2626import java .util .NavigableSet ;
2727import java .util .TreeSet ;
2828import java .util .concurrent .TimeUnit ;
29+ import java .util .function .Function ;
2930import java .util .stream .Collectors ;
3031
3132import com .google .common .annotations .VisibleForTesting ;
5152import org .apache .cassandra .db .partitions .PartitionIterators ;
5253import org .apache .cassandra .db .partitions .SingletonUnfilteredPartitionIterator ;
5354import org .apache .cassandra .db .partitions .UnfilteredPartitionIterator ;
55+ import org .apache .cassandra .db .rows .BaseRowIterator ;
5456import org .apache .cassandra .db .rows .Cell ;
5557import org .apache .cassandra .db .rows .Row ;
5658import org .apache .cassandra .db .rows .Rows ;
@@ -658,10 +660,26 @@ public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExe
658660 assert executionController != null && executionController .validForReadOn (cfs );
659661 Tracing .trace ("Executing single-partition query on {}" , cfs .name );
660662
661- return queryMemtableAndDiskInternal (cfs , executionController );
663+ Tracing .trace ("Acquiring sstable references" );
664+ ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
665+ return queryMemtableAndDiskInternal (cfs , view , null , executionController );
666+ }
667+
668+ public UnfilteredRowIterator queryMemtableAndDisk (ColumnFamilyStore cfs ,
669+ ColumnFamilyStore .ViewFragment view ,
670+ Function <CellSourceIdentifier , Transformation <BaseRowIterator <?>>> rowTransformer ,
671+ ReadExecutionController executionController )
672+ {
673+ assert executionController != null && executionController .validForReadOn (cfs );
674+ Tracing .trace ("Executing single-partition query on {}" , cfs .name );
675+
676+ return queryMemtableAndDiskInternal (cfs , view , rowTransformer , executionController );
662677 }
663678
664- private UnfilteredRowIterator queryMemtableAndDiskInternal (ColumnFamilyStore cfs , ReadExecutionController controller )
679+ private UnfilteredRowIterator queryMemtableAndDiskInternal (ColumnFamilyStore cfs ,
680+ ColumnFamilyStore .ViewFragment view ,
681+ Function <CellSourceIdentifier , Transformation <BaseRowIterator <?>>> rowTransformer ,
682+ ReadExecutionController controller )
665683 {
666684 /*
667685 * We have 2 main strategies:
@@ -685,11 +703,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
685703 && !queriesMulticellType ()
686704 && !controller .isTrackingRepairedStatus ())
687705 {
688- return queryMemtableAndSSTablesInTimestampOrder (cfs , (ClusteringIndexNamesFilter )clusteringIndexFilter (), controller );
706+ return queryMemtableAndSSTablesInTimestampOrder (cfs , view , rowTransformer , (ClusteringIndexNamesFilter )clusteringIndexFilter (), controller );
689707 }
690708
691- Tracing .trace ("Acquiring sstable references" );
692- ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
693709 view .sstables .sort (SSTableReader .maxTimestampDescending );
694710 ClusteringIndexFilter filter = clusteringIndexFilter ();
695711 long minTimestamp = Long .MAX_VALUE ;
@@ -708,6 +724,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
708724 if (memtable .getMinTimestamp () != Memtable .NO_MIN_TIMESTAMP )
709725 minTimestamp = Math .min (minTimestamp , memtable .getMinTimestamp ());
710726
727+ if (rowTransformer != null )
728+ iter = Transformation .apply (iter , rowTransformer .apply (memtable ));
729+
711730 // Memtable data is always considered unrepaired
712731 controller .updateMinOldestUnrepairedTombstone (memtable .getMinLocalDeletionTime ());
713732 inputCollector .addMemtableIterator (RTBoundValidator .validate (iter , RTBoundValidator .Stage .MEMTABLE , false ));
@@ -767,6 +786,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
767786 UnfilteredRowIterator iter = intersects ? makeRowIteratorWithLowerBound (cfs , sstable , metricsCollector )
768787 : makeRowIteratorWithSkippedNonStaticContent (cfs , sstable , metricsCollector );
769788
789+ if (rowTransformer != null )
790+ iter = Transformation .apply (iter , rowTransformer .apply (sstable .getId ()));
791+
770792 inputCollector .addSSTableIterator (sstable , iter );
771793 mostRecentPartitionTombstone = Math .max (mostRecentPartitionTombstone ,
772794 iter .partitionLevelDeletion ().markedForDeleteAt ());
@@ -789,6 +811,10 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
789811 {
790812 if (!sstable .isRepaired ())
791813 controller .updateMinOldestUnrepairedTombstone (sstable .getMinLocalDeletionTime ());
814+
815+ if (rowTransformer != null )
816+ iter = Transformation .apply (iter , rowTransformer .apply (sstable .getId ()));
817+
792818 inputCollector .addSSTableIterator (sstable , iter );
793819 includedDueToTombstones ++;
794820 mostRecentPartitionTombstone = Math .max (mostRecentPartitionTombstone ,
@@ -922,11 +948,8 @@ private boolean queriesMulticellType()
922948 * no collection or counters are included).
923949 * This method assumes the filter is a {@code ClusteringIndexNamesFilter}.
924950 */
925- private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder (ColumnFamilyStore cfs , ClusteringIndexNamesFilter filter , ReadExecutionController controller )
951+ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder (ColumnFamilyStore cfs , ColumnFamilyStore . ViewFragment view , Function < CellSourceIdentifier , Transformation < BaseRowIterator <?>>> rowTransformer , ClusteringIndexNamesFilter filter , ReadExecutionController controller )
926952 {
927- Tracing .trace ("Acquiring sstable references" );
928- ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
929-
930953 ImmutableBTreePartition result = null ;
931954 SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector ();
932955
@@ -938,7 +961,9 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
938961 if (iter == null )
939962 continue ;
940963
941- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .MEMTABLE , false ),
964+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (memtable ))
965+ : iter ;
966+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .MEMTABLE , false ),
942967 result ,
943968 filter ,
944969 false ,
@@ -993,7 +1018,10 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
9931018 }
9941019 else
9951020 {
996- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .SSTABLE , false ),
1021+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (sstable .getId ()))
1022+ : iter ;
1023+
1024+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .SSTABLE , false ),
9971025 result ,
9981026 filter ,
9991027 sstable .isRepaired (),
@@ -1008,8 +1036,9 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
10081036 {
10091037 if (iter .isEmpty ())
10101038 continue ;
1011-
1012- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .SSTABLE , false ),
1039+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (sstable .getId ()))
1040+ : iter ;
1041+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .SSTABLE , false ),
10131042 result ,
10141043 filter ,
10151044 sstable .isRepaired (),
0 commit comments