Skip to content

Commit 488ccaf

Browse files
committed
spanset: Add Overlaps() helper function
Analogous to how we have a spanset helper contains() that understands the special span representation: [x-eps,x). This commit adds Overlaps that expects the same span representation. Moreover, this commit makes this special span representation explicit by introducing a new type called `TrickySpan`.
1 parent 26c7441 commit 488ccaf

File tree

5 files changed

+233
-96
lines changed

5 files changed

+233
-96
lines changed

pkg/kv/kvserver/replica_eval_context_span.go

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool {
104104
func (rec SpanSetReplicaEvalContext) Desc() *roachpb.RangeDescriptor {
105105
desc := rec.i.Desc()
106106
rec.ss.AssertAllowed(spanset.SpanReadOnly,
107-
roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)},
107+
spanset.TrickySpan{Key: keys.RangeDescriptorKey(desc.StartKey)},
108108
)
109109
return desc
110110
}
@@ -145,7 +145,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord(
145145
ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp,
146146
) (bool, kvpb.TransactionAbortedReason) {
147147
rec.ss.AssertAllowed(spanset.SpanReadOnly,
148-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
148+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
149149
)
150150
return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS)
151151
}
@@ -157,7 +157,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
157157
ctx context.Context, txnID uuid.UUID, txnKey []byte,
158158
) hlc.Timestamp {
159159
rec.ss.AssertAllowed(spanset.SpanReadOnly,
160-
roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)},
160+
spanset.TrickySpan{Key: keys.TransactionKey(txnKey, txnID)},
161161
)
162162
return rec.i.MinTxnCommitTS(ctx, txnID, txnKey)
163163
}
@@ -167,7 +167,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS(
167167
// not be served.
168168
func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp {
169169
rec.ss.AssertAllowed(spanset.SpanReadOnly,
170-
roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
170+
spanset.TrickySpan{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
171171
)
172172
return rec.i.GetGCThreshold()
173173
}
@@ -191,7 +191,7 @@ func (rec SpanSetReplicaEvalContext) GetLastReplicaGCTimestamp(
191191
ctx context.Context,
192192
) (hlc.Timestamp, error) {
193193
if err := rec.ss.CheckAllowed(spanset.SpanReadOnly,
194-
roachpb.Span{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
194+
spanset.TrickySpan{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())},
195195
); err != nil {
196196
return hlc.Timestamp{}, err
197197
}
@@ -222,11 +222,11 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context)
222222
// To capture a read summary over the range, all keys must be latched for
223223
// writing to prevent any concurrent reads or writes.
224224
desc := rec.i.Desc()
225-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
225+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
226226
Key: keys.MakeRangeKeyPrefix(desc.StartKey),
227227
EndKey: keys.MakeRangeKeyPrefix(desc.EndKey),
228228
})
229-
rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{
229+
rec.ss.AssertAllowed(spanset.SpanReadWrite, spanset.TrickySpan{
230230
Key: desc.StartKey.AsRawKey(),
231231
EndKey: desc.EndKey.AsRawKey(),
232232
})

pkg/kv/kvserver/replica_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2732,7 +2732,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) {
27322732
{spanset.SpanReadWrite, roachpb.Key("b"), false},
27332733
{spanset.SpanReadWrite, roachpb.Key("d"), true},
27342734
} {
2735-
err := spans.CheckAllowed(tc.access, roachpb.Span{Key: tc.key})
2735+
err := spans.CheckAllowed(tc.access, spanset.TrickySpan{Key: tc.key})
27362736
if tc.expectAccess {
27372737
require.NoError(t, err)
27382738
} else {

pkg/kv/kvserver/spanset/batch.go

Lines changed: 25 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -77,15 +77,15 @@ func (i *MVCCIterator) Valid() (bool, error) {
7777
// SeekGE is part of the storage.MVCCIterator interface.
7878
func (i *MVCCIterator) SeekGE(key storage.MVCCKey) {
7979
i.i.SeekGE(key)
80-
i.checkAllowed(roachpb.Span{Key: key.Key}, true)
80+
i.checkAllowed(TrickySpan{Key: key.Key}, true)
8181
}
8282

8383
// SeekLT is part of the storage.MVCCIterator interface.
8484
func (i *MVCCIterator) SeekLT(key storage.MVCCKey) {
8585
i.i.SeekLT(key)
8686
// CheckAllowed{At} supports the span representation of [,key), which
8787
// corresponds to the span [key.Prev(),).
88-
i.checkAllowed(roachpb.Span{EndKey: key.Key}, true)
88+
i.checkAllowed(TrickySpan{EndKey: key.Key}, true)
8989
}
9090

9191
// Next is part of the storage.MVCCIterator interface.
@@ -117,12 +117,12 @@ func (i *MVCCIterator) checkAllowedCurrPosForward(errIfDisallowed bool) {
117117
// as long as the iterator itself is configured with proper boundaries.
118118
return
119119
}
120-
i.checkAllowedValidPos(roachpb.Span{Key: i.UnsafeKey().Key}, errIfDisallowed)
120+
i.checkAllowedValidPos(TrickySpan{Key: i.UnsafeKey().Key}, errIfDisallowed)
121121
}
122122

123123
// checkAllowed checks the provided span if the current iterator position is
124124
// valid.
125-
func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) {
125+
func (i *MVCCIterator) checkAllowed(span TrickySpan, errIfDisallowed bool) {
126126
i.invalid = false
127127
i.err = nil
128128
if ok, _ := i.i.Valid(); !ok {
@@ -134,7 +134,7 @@ func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) {
134134
i.checkAllowedValidPos(span, errIfDisallowed)
135135
}
136136

137-
func (i *MVCCIterator) checkAllowedValidPos(span roachpb.Span, errIfDisallowed bool) {
137+
func (i *MVCCIterator) checkAllowedValidPos(span TrickySpan, errIfDisallowed bool) {
138138
var err error
139139
if i.spansOnly {
140140
err = i.spans.CheckAllowed(SpanReadOnly, span)
@@ -213,11 +213,11 @@ func (i *MVCCIterator) FindSplitKey(
213213
start, end, minSplitKey roachpb.Key, targetSize int64,
214214
) (storage.MVCCKey, error) {
215215
if i.spansOnly {
216-
if err := i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil {
216+
if err := i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil {
217217
return storage.MVCCKey{}, err
218218
}
219219
} else {
220-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, i.ts); err != nil {
220+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, i.ts); err != nil {
221221
return storage.MVCCKey{}, err
222222
}
223223
}
@@ -261,10 +261,10 @@ func (i *EngineIterator) SeekEngineKeyGE(key storage.EngineKey) (valid bool, err
261261
}
262262
if key.IsMVCCKey() && !i.spansOnly {
263263
mvccKey, _ := key.ToMVCCKey()
264-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
264+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
265265
return false, err
266266
}
267-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
267+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
268268
return false, err
269269
}
270270
return valid, err
@@ -278,10 +278,10 @@ func (i *EngineIterator) SeekEngineKeyLT(key storage.EngineKey) (valid bool, err
278278
}
279279
if key.IsMVCCKey() && !i.spansOnly {
280280
mvccKey, _ := key.ToMVCCKey()
281-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
281+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
282282
return false, err
283283
}
284-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil {
284+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil {
285285
return false, err
286286
}
287287
return valid, err
@@ -313,7 +313,7 @@ func (i *EngineIterator) SeekEngineKeyGEWithLimit(
313313
if state != pebble.IterValid {
314314
return state, err
315315
}
316-
if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
316+
if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
317317
return pebble.IterExhausted, err
318318
}
319319
return state, err
@@ -327,7 +327,7 @@ func (i *EngineIterator) SeekEngineKeyLTWithLimit(
327327
if state != pebble.IterValid {
328328
return state, err
329329
}
330-
if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil {
330+
if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{EndKey: key.Key}); err != nil {
331331
return pebble.IterExhausted, err
332332
}
333333
return state, err
@@ -354,11 +354,11 @@ func (i *EngineIterator) checkKeyAllowed() (valid bool, err error) {
354354
}
355355
if key.IsMVCCKey() && !i.spansOnly {
356356
mvccKey, _ := key.ToMVCCKey()
357-
if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil {
357+
if err := i.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: mvccKey.Key}, i.ts); err != nil {
358358
// Invalid, but no error.
359359
return false, nil // nolint:returnerrcheck
360360
}
361-
} else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil {
361+
} else if err = i.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: key.Key}); err != nil {
362362
// Invalid, but no error.
363363
return false, nil // nolint:returnerrcheck
364364
}
@@ -469,11 +469,11 @@ func (s spanSetReader) MVCCIterate(
469469
f func(storage.MVCCKeyValue, storage.MVCCRangeKeyStack) error,
470470
) error {
471471
if s.spansOnly {
472-
if err := s.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil {
472+
if err := s.spans.CheckAllowed(SpanReadOnly, TrickySpan{Key: start, EndKey: end}); err != nil {
473473
return err
474474
}
475475
} else {
476-
if err := s.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil {
476+
if err := s.spans.CheckAllowedAt(SpanReadOnly, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil {
477477
return err
478478
}
479479
}
@@ -535,11 +535,11 @@ func (s spanSetWriter) ApplyBatchRepr(repr []byte, sync bool) error {
535535

536536
func (s spanSetWriter) checkAllowed(key roachpb.Key) error {
537537
if s.spansOnly {
538-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key}); err != nil {
538+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key}); err != nil {
539539
return err
540540
}
541541
} else {
542-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key}, s.ts); err != nil {
542+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key}, s.ts); err != nil {
543543
return err
544544
}
545545
}
@@ -561,7 +561,7 @@ func (s spanSetWriter) ClearUnversioned(key roachpb.Key, opts storage.ClearOptio
561561
}
562562

563563
func (s spanSetWriter) ClearEngineKey(key storage.EngineKey, opts storage.ClearOptions) error {
564-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
564+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
565565
return err
566566
}
567567
return s.w.ClearEngineKey(key, opts)
@@ -575,11 +575,11 @@ func (s spanSetWriter) SingleClearEngineKey(key storage.EngineKey) error {
575575

576576
func (s spanSetWriter) checkAllowedRange(start, end roachpb.Key) error {
577577
if s.spansOnly {
578-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}); err != nil {
578+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: start, EndKey: end}); err != nil {
579579
return err
580580
}
581581
} else {
582-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil {
582+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: start, EndKey: end}, s.ts); err != nil {
583583
return err
584584
}
585585
}
@@ -661,11 +661,11 @@ func (s spanSetWriter) ClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error {
661661

662662
func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error {
663663
if s.spansOnly {
664-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
664+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
665665
return err
666666
}
667667
} else {
668-
if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key.Key}, s.ts); err != nil {
668+
if err := s.spans.CheckAllowedAt(SpanReadWrite, TrickySpan{Key: key.Key}, s.ts); err != nil {
669669
return err
670670
}
671671
}
@@ -697,7 +697,7 @@ func (s spanSetWriter) PutEngineKey(key storage.EngineKey, value []byte) error {
697697
if !s.spansOnly {
698698
panic("cannot do timestamp checking for putting EngineKey")
699699
}
700-
if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil {
700+
if err := s.spans.CheckAllowed(SpanReadWrite, TrickySpan{Key: key.Key}); err != nil {
701701
return err
702702
}
703703
return s.w.PutEngineKey(key, value)

pkg/kv/kvserver/spanset/spanset.go

Lines changed: 40 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,11 @@ type Span struct {
7676
Timestamp hlc.Timestamp
7777
}
7878

79+
// TrickySpan represents a span that supports a special encoding where a nil
80+
// start key with a non-nil end key represents the point span:
81+
// [EndKey.Prev(), EndKey).
82+
type TrickySpan roachpb.Span
83+
7984
// SpanSet tracks the set of key spans touched by a command, broken into MVCC
8085
// and non-MVCC accesses. The set is divided into subsets for access type
8186
// (read-only or read/write) and key scope (local or global; used to facilitate
@@ -88,7 +93,7 @@ type SpanSet struct {
8893
// shouldn't be accessed (forbidden). This allows for complex pattern matching
8994
// like forbidding specific keys across all range IDs without enumerating them
9095
// explicitly.
91-
forbiddenSpansMatchers []func(roachpb.Span) error
96+
forbiddenSpansMatchers []func(TrickySpan) error
9297
allowUndeclared bool
9398
allowForbidden bool
9499
}
@@ -213,7 +218,7 @@ func (s *SpanSet) AddMVCC(access SpanAccess, span roachpb.Span, timestamp hlc.Ti
213218

214219
// AddForbiddenMatcher adds a forbidden span matcher. The matcher is a function
215220
// that is called for each span access to check if it should be forbidden.
216-
func (s *SpanSet) AddForbiddenMatcher(matcher func(roachpb.Span) error) {
221+
func (s *SpanSet) AddForbiddenMatcher(matcher func(TrickySpan) error) {
217222
s.forbiddenSpansMatchers = append(s.forbiddenSpansMatchers, matcher)
218223
}
219224

@@ -260,7 +265,7 @@ func (s *SpanSet) Intersects(other *SpanSet) bool {
260265
otherSpans := other.GetSpans(sa, ss)
261266
for _, span := range otherSpans {
262267
// If access is allowed, we must have an overlap.
263-
if err := s.CheckAllowed(sa, span.Span); err == nil {
268+
if err := s.CheckAllowed(sa, TrickySpan(span.Span)); err == nil {
264269
return true
265270
}
266271
}
@@ -272,7 +277,7 @@ func (s *SpanSet) Intersects(other *SpanSet) bool {
272277
// AssertAllowed calls CheckAllowed and fatals if the access is not allowed.
273278
// Timestamps associated with the spans in the spanset are not considered,
274279
// only the span boundaries are checked.
275-
func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) {
280+
func (s *SpanSet) AssertAllowed(access SpanAccess, span TrickySpan) {
276281
if err := s.CheckAllowed(access, span); err != nil {
277282
log.KvExec.Fatalf(context.TODO(), "%v", err)
278283
}
@@ -293,7 +298,7 @@ func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) {
293298
// fail at checking if read only access over the span [a-d) was requested. This
294299
// is also a problem if the added spans were read only and the spanset wasn't
295300
// already SortAndDedup-ed.
296-
func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error {
301+
func (s *SpanSet) CheckAllowed(access SpanAccess, span TrickySpan) error {
297302
return s.checkAllowed(access, span, func(_ SpanAccess, _ Span) bool {
298303
return true
299304
})
@@ -302,7 +307,7 @@ func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error {
302307
// CheckAllowedAt is like CheckAllowed, except it returns an error if the access
303308
// is not allowed over the given keyspan at the given timestamp.
304309
func (s *SpanSet) CheckAllowedAt(
305-
access SpanAccess, span roachpb.Span, timestamp hlc.Timestamp,
310+
access SpanAccess, span TrickySpan, timestamp hlc.Timestamp,
306311
) error {
307312
mvcc := !timestamp.IsEmpty()
308313
return s.checkAllowed(access, span, func(declAccess SpanAccess, declSpan Span) bool {
@@ -347,7 +352,7 @@ func (s *SpanSet) CheckAllowedAt(
347352
}
348353

349354
func (s *SpanSet) checkAllowed(
350-
access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool,
355+
access SpanAccess, span TrickySpan, check func(SpanAccess, Span) bool,
351356
) error {
352357
// Unless explicitly disabled, check if we access any forbidden spans.
353358
if !s.allowForbidden {
@@ -373,27 +378,20 @@ func (s *SpanSet) checkAllowed(
373378

374379
for ac := access; ac < NumSpanAccess; ac++ {
375380
for _, cur := range s.spans[ac][scope] {
376-
if contains(cur.Span, span) && check(ac, cur) {
381+
if Contains(cur.Span, span) && check(ac, cur) {
377382
return nil
378383
}
379384
}
380385
}
381386

382387
return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, s, debugutil.Stack())
383-
384-
return nil
385388
}
386389

387-
// contains returns whether s1 contains s2. Unlike Span.Contains, this function
388-
// supports spans with a nil start key and a non-nil end key (e.g. "[nil, c)").
389-
// In this form, s2.Key (inclusive) is considered to be the previous key to
390-
// s2.EndKey (exclusive).
391-
func contains(s1, s2 roachpb.Span) bool {
392-
if s2.Key != nil {
393-
// The common case.
394-
return s1.Contains(s2)
395-
}
396-
390+
// doesNormalSpanContainPointTrickySpan takes a normal span (s1) that has
391+
// both Key and EndKey not nil, and takes a tricky span s2, where the Key is
392+
// nil, which represents: [EndKey.Prev(), EndKey), and returns whether s1
393+
// contains s2 or not.
394+
func doesNormalSpanContainPointTrickySpan(s1 roachpb.Span, s2 TrickySpan) bool {
397395
// The following is equivalent to:
398396
// s1.Contains(roachpb.Span{Key: s2.EndKey.Prev()})
399397

@@ -404,6 +402,28 @@ func contains(s1, s2 roachpb.Span) bool {
404402
return s1.Key.Compare(s2.EndKey) < 0 && s1.EndKey.Compare(s2.EndKey) >= 0
405403
}
406404

405+
// Contains returns whether s1 contains s2, where s2 can be a TrickySpan.
406+
func Contains(s1 roachpb.Span, s2 TrickySpan) bool {
407+
if s2.Key != nil {
408+
// The common case: s2 is a regular span with a non-nil start key.
409+
return s1.Contains(roachpb.Span(s2))
410+
}
411+
412+
// s2 is a TrickySpan with nil Key and non-nil EndKey.
413+
return doesNormalSpanContainPointTrickySpan(s1, s2)
414+
}
415+
416+
// Overlaps returns whether s1 overlaps s2, where s2 can be a TrickySpan.
417+
func Overlaps(s1 roachpb.Span, s2 TrickySpan) bool {
418+
// The common case: both spans have non-nil start keys.
419+
if s2.Key != nil {
420+
return s1.Overlaps(roachpb.Span(s2))
421+
}
422+
423+
// s2 is a TrickySpan with nil Key and non-nil EndKey.
424+
return doesNormalSpanContainPointTrickySpan(s1, s2)
425+
}
426+
407427
// Validate returns an error if any spans that have been added to the set
408428
// are invalid.
409429
func (s *SpanSet) Validate() error {

0 commit comments

Comments
 (0)