-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
SinglePartitionReadCommand.java
1330 lines (1183 loc) · 62.4 KB
/
SinglePartitionReadCommand.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.cassandra.db;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.stream.Collectors;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.cassandra.cache.IRowCacheEntry;
import org.apache.cassandra.cache.RowCacheKey;
import org.apache.cassandra.cache.RowCacheSentinel;
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.lifecycle.*;
import org.apache.cassandra.db.filter.*;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.transform.RTBoundValidator;
import org.apache.cassandra.db.transform.Transformation;
import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.metrics.TableMetrics;
import org.apache.cassandra.net.MessageOut;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.schema.IndexMetadata;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.StorageProxy;
import org.apache.cassandra.service.pager.*;
import org.apache.cassandra.thrift.ThriftResultsMerger;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.transport.ProtocolVersion;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.SearchIterator;
import org.apache.cassandra.utils.btree.BTreeSet;
/**
* A read command that selects a (part of a) single partition.
*/
public class SinglePartitionReadCommand extends ReadCommand
{
protected static final SelectionDeserializer selectionDeserializer = new Deserializer();
private final DecoratedKey partitionKey;
private final ClusteringIndexFilter clusteringIndexFilter;
private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
private SinglePartitionReadCommand(boolean isDigest,
int digestVersion,
boolean isForThrift,
CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
RowFilter rowFilter,
DataLimits limits,
DecoratedKey partitionKey,
ClusteringIndexFilter clusteringIndexFilter,
IndexMetadata index)
{
super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
assert partitionKey.getPartitioner() == metadata.partitioner;
this.partitionKey = partitionKey;
this.clusteringIndexFilter = clusteringIndexFilter;
}
/**
* Creates a new read command on a single partition.
*
* @param isForThrift whether the query is for thrift or not.
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param columnFilter the column filter to use for the query.
* @param rowFilter the row filter to use for the query.
* @param limits the limits to use for the query.
* @param partitionKey the partition key for the partition to query.
* @param clusteringIndexFilter the clustering index filter to use for the query.
* @param indexMetadata explicitly specified index to use for the query
*
* @return a newly created read command.
*/
public static SinglePartitionReadCommand create(boolean isForThrift,
CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
RowFilter rowFilter,
DataLimits limits,
DecoratedKey partitionKey,
ClusteringIndexFilter clusteringIndexFilter,
IndexMetadata indexMetadata)
{
return new SinglePartitionReadCommand(false,
0,
isForThrift,
metadata,
nowInSec,
columnFilter,
rowFilter,
limits,
partitionKey,
clusteringIndexFilter,
indexMetadata);
}
/**
* Creates a new read command on a single partition.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param columnFilter the column filter to use for the query.
* @param rowFilter the row filter to use for the query.
* @param limits the limits to use for the query.
* @param partitionKey the partition key for the partition to query.
* @param clusteringIndexFilter the clustering index filter to use for the query.
*
* @return a newly created read command.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
RowFilter rowFilter,
DataLimits limits,
DecoratedKey partitionKey,
ClusteringIndexFilter clusteringIndexFilter)
{
return create(false, metadata, nowInSec, columnFilter, rowFilter, limits, partitionKey, clusteringIndexFilter);
}
/**
* Creates a new read command on a single partition.
*
* @param isForThrift whether the query is for thrift or not.
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param columnFilter the column filter to use for the query.
* @param rowFilter the row filter to use for the query.
* @param limits the limits to use for the query.
* @param partitionKey the partition key for the partition to query.
* @param clusteringIndexFilter the clustering index filter to use for the query.
*
* @return a newly created read command.
*/
public static SinglePartitionReadCommand create(boolean isForThrift,
CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
RowFilter rowFilter,
DataLimits limits,
DecoratedKey partitionKey,
ClusteringIndexFilter clusteringIndexFilter)
{
return create(isForThrift,
metadata,
nowInSec,
columnFilter,
rowFilter,
limits,
partitionKey,
clusteringIndexFilter,
findIndex(metadata, rowFilter));
}
/**
* Creates a new read command on a single partition.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param columnFilter the column filter to use for the query.
* @param filter the clustering index filter to use for the query.
*
* @return a newly created read command. The returned command will use no row filter and have no limits.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata,
int nowInSec,
DecoratedKey key,
ColumnFilter columnFilter,
ClusteringIndexFilter filter)
{
return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
}
/**
* Creates a new read command that queries a single partition in its entirety.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
*
* @return a newly created read command that queries all the rows of {@code key}.
*/
public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
{
return create(metadata, nowInSec, key, Slices.ALL);
}
/**
* Creates a new read command that queries a single partition in its entirety.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
*
* @return a newly created read command that queries all the rows of {@code key}.
*/
public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
{
return create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
}
/**
* Creates a new single partition slice command for the provided single slice.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param slice the slice of rows to query.
*
* @return a newly created read command that queries {@code slice} in {@code key}. The returned query will
* query every columns for the table (without limit or row filtering) and be in forward order.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slice slice)
{
return create(metadata, nowInSec, key, Slices.with(metadata.comparator, slice));
}
/**
* Creates a new single partition slice command for the provided slices.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param slices the slices of rows to query.
*
* @return a newly created read command that queries the {@code slices} in {@code key}. The returned query will
* query every columns for the table (without limit or row filtering) and be in forward order.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
{
ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
}
/**
* Creates a new single partition slice command for the provided slices.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param slices the slices of rows to query.
*
* @return a newly created read command that queries the {@code slices} in {@code key}. The returned query will
* query every columns for the table (without limit or row filtering) and be in forward order.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, ByteBuffer key, Slices slices)
{
return create(metadata, nowInSec, metadata.decorateKey(key), slices);
}
/**
* Creates a new single partition name command for the provided rows.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param names the clustering for the rows to query.
*
* @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
* query every columns (without limit or row filtering) and be in forward order.
*/
public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
{
ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
}
/**
* Creates a new single partition name command for the provided row.
*
* @param metadata the table to query.
* @param nowInSec the time in seconds to use are "now" for this query.
* @param key the partition key for the partition to query.
* @param name the clustering for the row to query.
*
* @return a newly created read command that queries {@code name} in {@code key}. The returned query will
* query every columns (without limit or row filtering).
*/
public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Clustering name)
{
return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
}
public SinglePartitionReadCommand copy()
{
return new SinglePartitionReadCommand(isDigestQuery(),
digestVersion(),
isForThrift(),
metadata(),
nowInSec(),
columnFilter(),
rowFilter(),
limits(),
partitionKey(),
clusteringIndexFilter(),
indexMetadata());
}
public SinglePartitionReadCommand copyAsDigestQuery()
{
return new SinglePartitionReadCommand(true,
digestVersion(),
isForThrift(),
metadata(),
nowInSec(),
columnFilter(),
rowFilter(),
limits(),
partitionKey(),
clusteringIndexFilter(),
indexMetadata());
}
public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
{
return new SinglePartitionReadCommand(isDigestQuery(),
digestVersion(),
isForThrift(),
metadata(),
nowInSec(),
columnFilter(),
rowFilter(),
newLimits,
partitionKey(),
clusteringIndexFilter(),
indexMetadata());
}
public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
{
return new SinglePartitionReadCommand(isDigestQuery(),
digestVersion(),
isForThrift(),
metadata(),
nowInSec(),
columnFilter(),
rowFilter(),
limits(),
partitionKey(),
filter,
indexMetadata());
}
static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
int digestVersion,
CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
DataLimits limits,
DecoratedKey partitionKey,
ClusteringIndexSliceFilter filter)
{
// messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
return new SinglePartitionReadCommand(isDigest,
digestVersion,
true,
metadata,
nowInSec,
columnFilter,
RowFilter.NONE,
limits,
partitionKey,
filter,
null);
}
static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
int digestVersion,
CFMetaData metadata,
int nowInSec,
ColumnFilter columnFilter,
DecoratedKey partitionKey,
ClusteringIndexNamesFilter filter)
{
// messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
}
public DecoratedKey partitionKey()
{
return partitionKey;
}
public ClusteringIndexFilter clusteringIndexFilter()
{
return clusteringIndexFilter;
}
public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key)
{
return clusteringIndexFilter;
}
public long getTimeout()
{
return DatabaseDescriptor.getReadRpcTimeout();
}
public boolean isReversed()
{
return clusteringIndexFilter.isReversed();
}
public boolean selectsKey(DecoratedKey key)
{
if (!this.partitionKey().equals(key))
return false;
return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
}
public boolean selectsClustering(DecoratedKey key, Clustering clustering)
{
if (clustering == Clustering.STATIC_CLUSTERING)
return !columnFilter().fetchedColumns().statics.isEmpty();
if (!clusteringIndexFilter().selects(clustering))
return false;
return rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering);
}
/**
* Returns a new command suitable to paging from the last returned row.
*
* @param lastReturned the last row returned by the previous page. The newly created command
* will only query row that comes after this (in query order). This can be {@code null} if this
* is the first page.
* @param limits the limits to use for the page to query.
*
* @return the newly create command.
*/
public SinglePartitionReadCommand forPaging(Clustering lastReturned, DataLimits limits)
{
// We shouldn't have set digest yet when reaching that point
assert !isDigestQuery();
return create(isForThrift(),
metadata(),
nowInSec(),
columnFilter(),
rowFilter(),
limits,
partitionKey(),
lastReturned == null ? clusteringIndexFilter() : clusteringIndexFilter.forPaging(metadata().comparator, lastReturned, false));
}
public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException
{
return StorageProxy.read(Group.one(this), consistency, clientState, queryStartNanoTime);
}
public SinglePartitionPager getPager(PagingState pagingState, ProtocolVersion protocolVersion)
{
return getPager(this, pagingState, protocolVersion);
}
private static SinglePartitionPager getPager(SinglePartitionReadCommand command, PagingState pagingState, ProtocolVersion protocolVersion)
{
return new SinglePartitionPager(command, pagingState, protocolVersion);
}
protected void recordLatency(TableMetrics metric, long latencyNanos)
{
metric.readLatency.addNano(latencyNanos);
}
@SuppressWarnings("resource") // we close the created iterator through closing the result of this method (and SingletonUnfilteredPartitionIterator ctor cannot fail)
protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
{
UnfilteredRowIterator partition = cfs.isRowCacheEnabled()
? getThroughCache(cfs, executionController)
: queryMemtableAndDisk(cfs, executionController);
return new SingletonUnfilteredPartitionIterator(partition, isForThrift());
}
/**
* Fetch the rows requested if in cache; if not, read it from disk and cache it.
* <p>
* If the partition is cached, and the filter given is within its bounds, we return
* from cache, otherwise from disk.
* <p>
* If the partition is is not cached, we figure out what filter is "biggest", read
* that from disk, then filter the result and either cache that or return it.
*/
private UnfilteredRowIterator getThroughCache(ColumnFamilyStore cfs, ReadExecutionController executionController)
{
assert !cfs.isIndex(); // CASSANDRA-5732
assert cfs.isRowCacheEnabled() : String.format("Row cache is not enabled on table [%s]", cfs.name);
RowCacheKey key = new RowCacheKey(metadata().ksAndCFName, partitionKey());
// Attempt a sentinel-read-cache sequence. if a write invalidates our sentinel, we'll return our
// (now potentially obsolete) data, but won't cache it. see CASSANDRA-3862
// TODO: don't evict entire partitions on writes (#2864)
IRowCacheEntry cached = CacheService.instance.rowCache.get(key);
if (cached != null)
{
if (cached instanceof RowCacheSentinel)
{
// Some other read is trying to cache the value, just do a normal non-caching read
Tracing.trace("Row cache miss (race)");
cfs.metric.rowCacheMiss.inc();
return queryMemtableAndDisk(cfs, executionController);
}
CachedPartition cachedPartition = (CachedPartition)cached;
if (cfs.isFilterFullyCoveredBy(clusteringIndexFilter(), limits(), cachedPartition, nowInSec()))
{
cfs.metric.rowCacheHit.inc();
Tracing.trace("Row cache hit");
UnfilteredRowIterator unfilteredRowIterator = clusteringIndexFilter().getUnfilteredRowIterator(columnFilter(), cachedPartition);
cfs.metric.updateSSTableIterated(0);
return unfilteredRowIterator;
}
cfs.metric.rowCacheHitOutOfRange.inc();
Tracing.trace("Ignoring row cache as cached value could not satisfy query");
return queryMemtableAndDisk(cfs, executionController);
}
cfs.metric.rowCacheMiss.inc();
Tracing.trace("Row cache miss");
// Note that on tables with no clustering keys, any positive value of
// rowsToCache implies caching the full partition
boolean cacheFullPartitions = metadata().clusteringColumns().size() > 0 ?
metadata().params.caching.cacheAllRows() :
metadata().params.caching.cacheRows();
// To be able to cache what we read, what we read must at least covers what the cache holds, that
// is the 'rowsToCache' first rows of the partition. We could read those 'rowsToCache' first rows
// systematically, but we'd have to "extend" that to whatever is needed for the user query that the
// 'rowsToCache' first rows don't cover and it's not trivial with our existing filters. So currently
// we settle for caching what we read only if the user query does query the head of the partition since
// that's the common case of when we'll be able to use the cache anyway. One exception is if we cache
// full partitions, in which case we just always read it all and cache.
if (cacheFullPartitions || clusteringIndexFilter().isHeadFilter())
{
RowCacheSentinel sentinel = new RowCacheSentinel();
boolean sentinelSuccess = CacheService.instance.rowCache.putIfAbsent(key, sentinel);
boolean sentinelReplaced = false;
try
{
final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
final boolean enforceStrictLiveness = metadata().enforceStrictLiveness();
@SuppressWarnings("resource") // we close on exception or upon closing the result of this method
UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, executionController);
try
{
// Use a custom iterator instead of DataLimits to avoid stopping the original iterator
UnfilteredRowIterator toCacheIterator = new WrappingUnfilteredRowIterator(iter)
{
private int rowsCounted = 0;
@Override
public boolean hasNext()
{
return rowsCounted < rowsToCache && super.hasNext();
}
@Override
public Unfiltered next()
{
Unfiltered unfiltered = super.next();
if (unfiltered.isRow())
{
Row row = (Row) unfiltered;
if (row.hasLiveData(nowInSec(), enforceStrictLiveness))
rowsCounted++;
}
return unfiltered;
}
};
// We want to cache only rowsToCache rows
CachedPartition toCache = CachedBTreePartition.create(toCacheIterator, nowInSec());
if (sentinelSuccess && !toCache.isEmpty())
{
Tracing.trace("Caching {} rows", toCache.rowCount());
CacheService.instance.rowCache.replace(key, sentinel, toCache);
// Whether or not the previous replace has worked, our sentinel is not in the cache anymore
sentinelReplaced = true;
}
// We then re-filter out what this query wants.
// Note that in the case where we don't cache full partitions, it's possible that the current query is interested in more
// than what we've cached, so we can't just use toCache.
UnfilteredRowIterator cacheIterator = clusteringIndexFilter().getUnfilteredRowIterator(columnFilter(), toCache);
if (cacheFullPartitions)
{
// Everything is guaranteed to be in 'toCache', we're done with 'iter'
assert !iter.hasNext();
iter.close();
return cacheIterator;
}
return UnfilteredRowIterators.concat(cacheIterator, clusteringIndexFilter().filterNotIndexed(columnFilter(), iter));
}
catch (RuntimeException | Error e)
{
iter.close();
throw e;
}
}
finally
{
if (sentinelSuccess && !sentinelReplaced)
cfs.invalidateCachedPartition(key);
}
}
Tracing.trace("Fetching data but not populating cache as query does not query from the start of the partition");
return queryMemtableAndDisk(cfs, executionController);
}
/**
* Queries both memtable and sstables to fetch the result of this query.
* <p>
* Please note that this method:
* 1) does not check the row cache.
* 2) does not apply the query limit, nor the row filter (and so ignore 2ndary indexes).
* Those are applied in {@link ReadCommand#executeLocally}.
* 3) does not record some of the read metrics (latency, scanned cells histograms) nor
* throws TombstoneOverwhelmingException.
* It is publicly exposed because there is a few places where that is exactly what we want,
* but it should be used only where you know you don't need thoses things.
* <p>
* Also note that one must have created a {@code ReadExecutionController} on the queried table and we require it as
* a parameter to enforce that fact, even though it's not explicitlly used by the method.
*/
public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExecutionController executionController)
{
assert executionController != null && executionController.validForReadOn(cfs);
Tracing.trace("Executing single-partition query on {}", cfs.name);
return queryMemtableAndDiskInternal(cfs);
}
@Override
protected int oldestUnrepairedTombstone()
{
return oldestUnrepairedTombstone;
}
private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs)
{
/*
* We have 2 main strategies:
* 1) We query memtables and sstables simulateneously. This is our most generic strategy and the one we use
* unless we have a names filter that we know we can optimize futher.
* 2) If we have a name filter (so we query specific rows), we can make a bet: that all column for all queried row
* will have data in the most recent sstable(s), thus saving us from reading older ones. This does imply we
* have a way to guarantee we have all the data for what is queried, which is only possible for name queries
* and if we have neither non-frozen collections/UDTs nor counters (indeed, for a non-frozen collection or UDT,
* we can't guarantee an older sstable won't have some elements that weren't in the most recent sstables,
* and counters are intrinsically a collection of shards and so have the same problem).
*/
if (clusteringIndexFilter() instanceof ClusteringIndexNamesFilter && !queriesMulticellType())
return queryMemtableAndSSTablesInTimestampOrder(cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter());
Tracing.trace("Acquiring sstable references");
ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey()));
List<UnfilteredRowIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
ClusteringIndexFilter filter = clusteringIndexFilter();
long minTimestamp = Long.MAX_VALUE;
try
{
for (Memtable memtable : view.memtables)
{
Partition partition = memtable.getPartition(partitionKey());
if (partition == null)
continue;
minTimestamp = Math.min(minTimestamp, memtable.getMinTimestamp());
@SuppressWarnings("resource") // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator
UnfilteredRowIterator iter = filter.getUnfilteredRowIterator(columnFilter(), partition);
if (isForThrift())
iter = ThriftResultsMerger.maybeWrap(iter, nowInSec());
oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, partition.stats().minLocalDeletionTime);
iterators.add(RTBoundValidator.validate(iter, RTBoundValidator.Stage.MEMTABLE, false));
}
/*
* We can't eliminate full sstables based on the timestamp of what we've already read like
* in collectTimeOrderedData, but we still want to eliminate sstable whose maxTimestamp < mostRecentTombstone
* we've read. We still rely on the sstable ordering by maxTimestamp since if
* maxTimestamp_s1 < maxTimestamp_s0,
* we're guaranteed that s1 cannot have a row tombstone such that
* timestamp(tombstone) > maxTimestamp_s0
* since we necessarily have
* timestamp(tombstone) <= maxTimestamp_s1
* In other words, iterating in descending maxTimestamp order allow to do our mostRecentPartitionTombstone
* elimination in one pass, and minimize the number of sstables for which we read a partition tombstone.
*/
Collections.sort(view.sstables, SSTableReader.maxTimestampDescending);
long mostRecentPartitionTombstone = Long.MIN_VALUE;
int nonIntersectingSSTables = 0;
int includedDueToTombstones = 0;
SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector();
for (SSTableReader sstable : view.sstables)
{
// if we've already seen a partition tombstone with a timestamp greater
// than the most recent update to this sstable, we can skip it
if (sstable.getMaxTimestamp() < mostRecentPartitionTombstone)
break;
if (shouldInclude(sstable))
{
if (!sstable.isRepaired())
oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
// 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator
@SuppressWarnings("resource")
UnfilteredRowIterator iter = makeIterator(cfs, sstable, true, metricsCollector);
iterators.add(iter);
mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone,
iter.partitionLevelDeletion().markedForDeleteAt());
}
else
{
nonIntersectingSSTables++;
// sstable contains no tombstone if maxLocalDeletionTime == Integer.MAX_VALUE, so we can safely skip those entirely
if (sstable.mayHaveTombstones())
{
// 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator
@SuppressWarnings("resource")
UnfilteredRowIterator iter = makeIterator(cfs, sstable, true, metricsCollector);
// if the sstable contains a partition delete, then we must include it regardless of whether it
// shadows any other data seen locally as we can't guarantee that other replicas have seen it
if (!iter.partitionLevelDeletion().isLive())
{
if (!sstable.isRepaired())
oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime());
iterators.add(iter);
includedDueToTombstones++;
mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone,
iter.partitionLevelDeletion().markedForDeleteAt());
}
else
{
iter.close();
}
}
}
}
if (Tracing.isTracing())
Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {} due to tombstones",
nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones);
if (iterators.isEmpty())
return EmptyIterators.unfilteredRow(cfs.metadata, partitionKey(), filter.isReversed());
StorageHook.instance.reportRead(cfs.metadata.cfId, partitionKey());
return withSSTablesIterated(iterators, cfs.metric, metricsCollector);
}
catch (RuntimeException | Error e)
{
try
{
FBUtilities.closeAll(iterators);
}
catch (Exception suppressed)
{
e.addSuppressed(suppressed);
}
throw e;
}
}
private boolean shouldInclude(SSTableReader sstable)
{
// If some static columns are queried, we should always include the sstable: the clustering values stats of the sstable
// don't tell us if the sstable contains static values in particular.
// TODO: we could record if a sstable contains any static value at all.
if (!columnFilter().fetchedColumns().statics.isEmpty())
return true;
return clusteringIndexFilter().shouldInclude(sstable);
}
private UnfilteredRowIteratorWithLowerBound makeIterator(ColumnFamilyStore cfs,
final SSTableReader sstable,
boolean applyThriftTransformation,
SSTableReadsListener listener)
{
return StorageHook.instance.makeRowIteratorWithLowerBound(cfs,
partitionKey(),
sstable,
clusteringIndexFilter(),
columnFilter(),
isForThrift(),
nowInSec(),
applyThriftTransformation,
listener);
}
/**
* Return a wrapped iterator that when closed will update the sstables iterated and READ sample metrics.
* Note that we cannot use the Transformations framework because they greedily get the static row, which
* would cause all iterators to be initialized and hence all sstables to be accessed.
*/
private UnfilteredRowIterator withSSTablesIterated(List<UnfilteredRowIterator> iterators,
TableMetrics metrics,
SSTableReadMetricsCollector metricsCollector)
{
@SuppressWarnings("resource") // Closed through the closing of the result of the caller method.
UnfilteredRowIterator merged = UnfilteredRowIterators.merge(iterators, nowInSec());
if (!merged.isEmpty())
{
DecoratedKey key = merged.partitionKey();
metrics.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), key.hashCode(), 1);
}
class UpdateSstablesIterated extends Transformation
{
public void onPartitionClose()
{
int mergedSSTablesIterated = metricsCollector.getMergedSSTables();
metrics.updateSSTableIterated(mergedSSTablesIterated);
Tracing.trace("Merged data from memtables and {} sstables", mergedSSTablesIterated);
}
};
return Transformation.apply(merged, new UpdateSstablesIterated());
}
private boolean queriesMulticellType()
{
for (ColumnDefinition column : columnFilter().fetchedColumns())
{
if (column.type.isMultiCell() || column.type.isCounter())
return true;
}
return false;
}
/**
* Do a read by querying the memtable(s) first, and then each relevant sstables sequentially by order of the sstable
* max timestamp.
*
* This is used for names query in the hope of only having to query the 1 or 2 most recent query and then knowing nothing
* more recent could be in the older sstables (which we can only guarantee if we know exactly which row we queries, and if
* no collection or counters are included).
* This method assumes the filter is a {@code ClusteringIndexNamesFilter}.
*/
private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter)
{
Tracing.trace("Acquiring sstable references");
ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey()));
ImmutableBTreePartition result = null;
Tracing.trace("Merging memtable contents");
for (Memtable memtable : view.memtables)
{
Partition partition = memtable.getPartition(partitionKey());
if (partition == null)
continue;
try (UnfilteredRowIterator iter = filter.getUnfilteredRowIterator(columnFilter(), partition))
{
if (iter.isEmpty())
continue;
result = add(
RTBoundValidator.validate(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, nowInSec()) : iter, RTBoundValidator.Stage.MEMTABLE, false),
result,
filter,
false
);
}
}
/* add the SSTables on disk */
Collections.sort(view.sstables, SSTableReader.maxTimestampDescending);
// read sorted sstables
SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector();
for (SSTableReader sstable : view.sstables)
{
// if we've already seen a partition tombstone with a timestamp greater
// than the most recent update to this sstable, we're done, since the rest of the sstables
// will also be older
if (result != null && sstable.getMaxTimestamp() < result.partitionLevelDeletion().markedForDeleteAt())
break;
long currentMaxTs = sstable.getMaxTimestamp();
filter = reduceFilter(filter, result, currentMaxTs);
if (filter == null)
break;
if (!shouldInclude(sstable))
{
// This mean that nothing queried by the filter can be in the sstable. One exception is the top-level partition deletion
// however: if it is set, it impacts everything and must be included. Getting that top-level partition deletion costs us
// some seek in general however (unless the partition is indexed and is in the key cache), so we first check if the sstable
// has any tombstone at all as a shortcut.
if (!sstable.mayHaveTombstones())
continue; // no tombstone at all, we can skip that sstable
// We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs,
sstable,
partitionKey(),
filter.getSlices(metadata()),
columnFilter(),
filter.isReversed(),
isForThrift(),
metricsCollector))
{
if (!iter.partitionLevelDeletion().isLive())
{
result = add(
UnfilteredRowIterators.noRowsIterator(iter.metadata(),
iter.partitionKey(),
Rows.EMPTY_STATIC_ROW,
iter.partitionLevelDeletion(),
filter.isReversed()),
result,
filter,
sstable.isRepaired()
);
}
else
{
result = add(
RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false),
result,
filter,
sstable.isRepaired()
);
}
}
continue;
}
try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs,
sstable,
partitionKey(),
filter.getSlices(metadata()),
columnFilter(),
filter.isReversed(),
isForThrift(),
metricsCollector))
{
if (iter.isEmpty())
continue;
result = add(
RTBoundValidator.validate(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, nowInSec()) : iter, RTBoundValidator.Stage.SSTABLE, false),
result,
filter,
sstable.isRepaired()
);