5656import org .apache .cassandra .db .partitions .PartitionIterators ;
5757import org .apache .cassandra .db .partitions .SingletonUnfilteredPartitionIterator ;
5858import org .apache .cassandra .db .partitions .UnfilteredPartitionIterator ;
59+ import org .apache .cassandra .db .rows .BaseRowIterator ;
5960import org .apache .cassandra .db .rows .Cell ;
6061import org .apache .cassandra .db .rows .Row ;
6162import org .apache .cassandra .db .rows .Rows ;
@@ -726,10 +727,26 @@ public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExe
726727 assert executionController != null && executionController .validForReadOn (cfs );
727728 Tracing .trace ("Executing single-partition query on {}" , cfs .name );
728729
729- return queryMemtableAndDiskInternal (cfs , executionController );
730+ Tracing .trace ("Acquiring sstable references" );
731+ ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
732+ return queryMemtableAndDiskInternal (cfs , view , null , executionController );
733+ }
734+
735+ public UnfilteredRowIterator queryMemtableAndDisk (ColumnFamilyStore cfs ,
736+ ColumnFamilyStore .ViewFragment view ,
737+ Function <CellSourceIdentifier , Transformation <BaseRowIterator <?>>> rowTransformer ,
738+ ReadExecutionController executionController )
739+ {
740+ assert executionController != null && executionController .validForReadOn (cfs );
741+ Tracing .trace ("Executing single-partition query on {}" , cfs .name );
742+
743+ return queryMemtableAndDiskInternal (cfs , view , rowTransformer , executionController );
730744 }
731745
732- private UnfilteredRowIterator queryMemtableAndDiskInternal (ColumnFamilyStore cfs , ReadExecutionController controller )
746+ private UnfilteredRowIterator queryMemtableAndDiskInternal (ColumnFamilyStore cfs ,
747+ ColumnFamilyStore .ViewFragment view ,
748+ Function <CellSourceIdentifier , Transformation <BaseRowIterator <?>>> rowTransformer ,
749+ ReadExecutionController controller )
733750 {
734751 /*
735752 * We have 2 main strategies:
@@ -753,11 +770,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
753770 && !queriesMulticellType ()
754771 && !controller .isTrackingRepairedStatus ())
755772 {
756- return queryMemtableAndSSTablesInTimestampOrder (cfs , (ClusteringIndexNamesFilter )clusteringIndexFilter (), controller );
773+ return queryMemtableAndSSTablesInTimestampOrder (cfs , view , rowTransformer , (ClusteringIndexNamesFilter )clusteringIndexFilter (), controller );
757774 }
758775
759- Tracing .trace ("Acquiring sstable references" );
760- ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
761776 view .sstables .sort (SSTableReader .maxTimestampDescending );
762777 ClusteringIndexFilter filter = clusteringIndexFilter ();
763778 long minTimestamp = Long .MAX_VALUE ;
@@ -776,6 +791,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
776791 if (memtable .getMinTimestamp () != Memtable .NO_MIN_TIMESTAMP )
777792 minTimestamp = Math .min (minTimestamp , memtable .getMinTimestamp ());
778793
794+ if (rowTransformer != null )
795+ iter = Transformation .apply (iter , rowTransformer .apply (memtable ));
796+
779797 // Memtable data is always considered unrepaired
780798 controller .updateMinOldestUnrepairedTombstone (memtable .getMinLocalDeletionTime ());
781799 inputCollector .addMemtableIterator (RTBoundValidator .validate (iter , RTBoundValidator .Stage .MEMTABLE , false ));
@@ -835,6 +853,9 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
835853 UnfilteredRowIterator iter = intersects ? makeRowIteratorWithLowerBound (cfs , sstable , metricsCollector )
836854 : makeRowIteratorWithSkippedNonStaticContent (cfs , sstable , metricsCollector );
837855
856+ if (rowTransformer != null )
857+ iter = Transformation .apply (iter , rowTransformer .apply (sstable .getId ()));
858+
838859 inputCollector .addSSTableIterator (sstable , iter );
839860 mostRecentPartitionTombstone = Math .max (mostRecentPartitionTombstone ,
840861 iter .partitionLevelDeletion ().markedForDeleteAt ());
@@ -857,6 +878,10 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs
857878 {
858879 if (!sstable .isRepaired ())
859880 controller .updateMinOldestUnrepairedTombstone (sstable .getMinLocalDeletionTime ());
881+
882+ if (rowTransformer != null )
883+ iter = Transformation .apply (iter , rowTransformer .apply (sstable .getId ()));
884+
860885 inputCollector .addSSTableIterator (sstable , iter );
861886 includedDueToTombstones ++;
862887 mostRecentPartitionTombstone = Math .max (mostRecentPartitionTombstone ,
@@ -996,11 +1021,8 @@ private boolean queriesMulticellType()
9961021 * no collection or counters are included).
9971022 * This method assumes the filter is a {@code ClusteringIndexNamesFilter}.
9981023 */
999- private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder (ColumnFamilyStore cfs , ClusteringIndexNamesFilter filter , ReadExecutionController controller )
1024+ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder (ColumnFamilyStore cfs , ColumnFamilyStore . ViewFragment view , Function < CellSourceIdentifier , Transformation < BaseRowIterator <?>>> rowTransformer , ClusteringIndexNamesFilter filter , ReadExecutionController controller )
10001025 {
1001- Tracing .trace ("Acquiring sstable references" );
1002- ColumnFamilyStore .ViewFragment view = cfs .select (View .select (SSTableSet .LIVE , partitionKey ()));
1003-
10041026 ImmutableBTreePartition result = null ;
10051027 SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector ();
10061028
@@ -1012,7 +1034,9 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
10121034 if (iter == null )
10131035 continue ;
10141036
1015- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .MEMTABLE , false ),
1037+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (memtable ))
1038+ : iter ;
1039+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .MEMTABLE , false ),
10161040 result ,
10171041 filter ,
10181042 false ,
@@ -1067,7 +1091,10 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
10671091 }
10681092 else
10691093 {
1070- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .SSTABLE , false ),
1094+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (sstable .getId ()))
1095+ : iter ;
1096+
1097+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .SSTABLE , false ),
10711098 result ,
10721099 filter ,
10731100 sstable .isRepaired (),
@@ -1082,8 +1109,9 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
10821109 {
10831110 if (iter .isEmpty ())
10841111 continue ;
1085-
1086- result = add (RTBoundValidator .validate (iter , RTBoundValidator .Stage .SSTABLE , false ),
1112+ UnfilteredRowIterator wrapped = rowTransformer != null ? Transformation .apply (iter , rowTransformer .apply (sstable .getId ()))
1113+ : iter ;
1114+ result = add (RTBoundValidator .validate (wrapped , RTBoundValidator .Stage .SSTABLE , false ),
10871115 result ,
10881116 filter ,
10891117 sstable .isRepaired (),
0 commit comments