2020import com .datastax .oss .driver .api .core .CqlIdentifier ;
2121import com .datastax .oss .driver .api .core .cql .SimpleStatement ;
2222import com .datastax .oss .driver .api .core .cql .SimpleStatementBuilder ;
23+ import com .datastax .oss .driver .api .core .data .CqlVector ;
2324import com .datastax .oss .driver .api .core .metadata .schema .ClusteringOrder ;
2425import com .datastax .oss .driver .api .querybuilder .BindMarker ;
26+ import com .datastax .oss .driver .api .querybuilder .QueryBuilder ;
2527import com .datastax .oss .driver .api .querybuilder .relation .Relation ;
2628import com .datastax .oss .driver .api .querybuilder .select .Select ;
2729import com .datastax .oss .driver .api .querybuilder .select .SelectFrom ;
@@ -49,6 +51,7 @@ public class DefaultSelect implements SelectFrom, Select {
4951 private final ImmutableList <Relation > relations ;
5052 private final ImmutableList <Selector > groupByClauses ;
5153 private final ImmutableMap <CqlIdentifier , ClusteringOrder > orderings ;
54+ private final Ann ann ;
5255 private final Object limit ;
5356 private final Object perPartitionLimit ;
5457 private final boolean allowsFiltering ;
@@ -65,6 +68,7 @@ public DefaultSelect(@Nullable CqlIdentifier keyspace, @NonNull CqlIdentifier ta
6568 ImmutableMap .of (),
6669 null ,
6770 null ,
71+ null ,
6872 false );
6973 }
7074
@@ -74,6 +78,8 @@ public DefaultSelect(@Nullable CqlIdentifier keyspace, @NonNull CqlIdentifier ta
7478 * @param selectors if it contains {@link AllSelector#INSTANCE}, that must be the only element.
7579 * This isn't re-checked because methods that call this constructor internally already do it,
7680 * make sure you do it yourself.
81+ * @param ann Approximate nearest neighbor. ANN ordering does not support secondary ordering or
82+ * ASC order.
7783 */
7884 public DefaultSelect (
7985 @ Nullable CqlIdentifier keyspace ,
@@ -84,6 +90,7 @@ public DefaultSelect(
8490 @ NonNull ImmutableList <Relation > relations ,
8591 @ NonNull ImmutableList <Selector > groupByClauses ,
8692 @ NonNull ImmutableMap <CqlIdentifier , ClusteringOrder > orderings ,
93+ @ Nullable Ann ann ,
8794 @ Nullable Object limit ,
8895 @ Nullable Object perPartitionLimit ,
8996 boolean allowsFiltering ) {
@@ -94,6 +101,9 @@ public DefaultSelect(
94101 || (limit instanceof Integer && (Integer ) limit > 0 )
95102 || limit instanceof BindMarker ,
96103 "limit must be a strictly positive integer or a bind marker" );
104+ Preconditions .checkArgument (
105+ orderings .isEmpty () || ann == null , "ANN ordering does not support secondary ordering" );
106+ this .ann = ann ;
97107 this .keyspace = keyspace ;
98108 this .table = table ;
99109 this .isJson = isJson ;
@@ -117,6 +127,7 @@ public SelectFrom json() {
117127 relations ,
118128 groupByClauses ,
119129 orderings ,
130+ ann ,
120131 limit ,
121132 perPartitionLimit ,
122133 allowsFiltering );
@@ -134,6 +145,7 @@ public SelectFrom distinct() {
134145 relations ,
135146 groupByClauses ,
136147 orderings ,
148+ ann ,
137149 limit ,
138150 perPartitionLimit ,
139151 allowsFiltering );
@@ -193,6 +205,7 @@ public Select withSelectors(@NonNull ImmutableList<Selector> newSelectors) {
193205 relations ,
194206 groupByClauses ,
195207 orderings ,
208+ ann ,
196209 limit ,
197210 perPartitionLimit ,
198211 allowsFiltering );
@@ -221,6 +234,7 @@ public Select withRelations(@NonNull ImmutableList<Relation> newRelations) {
221234 newRelations ,
222235 groupByClauses ,
223236 orderings ,
237+ ann ,
224238 limit ,
225239 perPartitionLimit ,
226240 allowsFiltering );
@@ -249,6 +263,7 @@ public Select withGroupByClauses(@NonNull ImmutableList<Selector> newGroupByClau
249263 relations ,
250264 newGroupByClauses ,
251265 orderings ,
266+ ann ,
252267 limit ,
253268 perPartitionLimit ,
254269 allowsFiltering );
@@ -260,6 +275,18 @@ public Select orderBy(@NonNull CqlIdentifier columnId, @NonNull ClusteringOrder
260275 return withOrderings (ImmutableCollections .append (orderings , columnId , order ));
261276 }
262277
278+ @ NonNull
279+ @ Override
280+ public Select orderByAnnOf (@ NonNull String columnName , @ NonNull CqlVector <?> ann ) {
281+ return withAnn (new Ann (CqlIdentifier .fromCql (columnName ), ann ));
282+ }
283+
284+ @ NonNull
285+ @ Override
286+ public Select orderByAnnOf (@ NonNull CqlIdentifier columnId , @ NonNull CqlVector <?> ann ) {
287+ return withAnn (new Ann (columnId , ann ));
288+ }
289+
263290 @ NonNull
264291 @ Override
265292 public Select orderByIds (@ NonNull Map <CqlIdentifier , ClusteringOrder > newOrderings ) {
@@ -277,6 +304,24 @@ public Select withOrderings(@NonNull ImmutableMap<CqlIdentifier, ClusteringOrder
277304 relations ,
278305 groupByClauses ,
279306 newOrderings ,
307+ ann ,
308+ limit ,
309+ perPartitionLimit ,
310+ allowsFiltering );
311+ }
312+
313+ @ NonNull
314+ Select withAnn (@ NonNull Ann ann ) {
315+ return new DefaultSelect (
316+ keyspace ,
317+ table ,
318+ isJson ,
319+ isDistinct ,
320+ selectors ,
321+ relations ,
322+ groupByClauses ,
323+ orderings ,
324+ ann ,
280325 limit ,
281326 perPartitionLimit ,
282327 allowsFiltering );
@@ -295,6 +340,7 @@ public Select limit(int limit) {
295340 relations ,
296341 groupByClauses ,
297342 orderings ,
343+ ann ,
298344 limit ,
299345 perPartitionLimit ,
300346 allowsFiltering );
@@ -312,6 +358,7 @@ public Select limit(@Nullable BindMarker bindMarker) {
312358 relations ,
313359 groupByClauses ,
314360 orderings ,
361+ ann ,
315362 bindMarker ,
316363 perPartitionLimit ,
317364 allowsFiltering );
@@ -331,6 +378,7 @@ public Select perPartitionLimit(int perPartitionLimit) {
331378 relations ,
332379 groupByClauses ,
333380 orderings ,
381+ ann ,
334382 limit ,
335383 perPartitionLimit ,
336384 allowsFiltering );
@@ -348,6 +396,7 @@ public Select perPartitionLimit(@Nullable BindMarker bindMarker) {
348396 relations ,
349397 groupByClauses ,
350398 orderings ,
399+ ann ,
351400 limit ,
352401 bindMarker ,
353402 allowsFiltering );
@@ -365,6 +414,7 @@ public Select allowFiltering() {
365414 relations ,
366415 groupByClauses ,
367416 orderings ,
417+ ann ,
368418 limit ,
369419 perPartitionLimit ,
370420 true );
@@ -391,15 +441,20 @@ public String asCql() {
391441 CqlHelper .append (relations , builder , " WHERE " , " AND " , null );
392442 CqlHelper .append (groupByClauses , builder , " GROUP BY " , "," , null );
393443
394- boolean first = true ;
395- for (Map .Entry <CqlIdentifier , ClusteringOrder > entry : orderings .entrySet ()) {
396- if (first ) {
397- builder .append (" ORDER BY " );
398- first = false ;
399- } else {
400- builder .append ("," );
444+ if (ann != null ) {
445+ builder .append (" ORDER BY " ).append (this .ann .columnId .asCql (true )).append (" ANN OF " );
446+ QueryBuilder .literal (ann .vector ).appendTo (builder );
447+ } else {
448+ boolean first = true ;
449+ for (Map .Entry <CqlIdentifier , ClusteringOrder > entry : orderings .entrySet ()) {
450+ if (first ) {
451+ builder .append (" ORDER BY " );
452+ first = false ;
453+ } else {
454+ builder .append ("," );
455+ }
456+ builder .append (entry .getKey ().asCql (true )).append (" " ).append (entry .getValue ().name ());
401457 }
402- builder .append (entry .getKey ().asCql (true )).append (" " ).append (entry .getValue ().name ());
403458 }
404459
405460 if (limit != null ) {
@@ -499,6 +554,11 @@ public Object getLimit() {
499554 return limit ;
500555 }
501556
557+ @ Nullable
558+ public Ann getAnn () {
559+ return ann ;
560+ }
561+
502562 @ Nullable
503563 public Object getPerPartitionLimit () {
504564 return perPartitionLimit ;
@@ -512,4 +572,14 @@ public boolean allowsFiltering() {
512572 public String toString () {
513573 return asCql ();
514574 }
575+
576+ public static class Ann {
577+ private final CqlVector <?> vector ;
578+ private final CqlIdentifier columnId ;
579+
580+ private Ann (CqlIdentifier columnId , CqlVector <? extends Number > vector ) {
581+ this .vector = vector ;
582+ this .columnId = columnId ;
583+ }
584+ }
515585}
0 commit comments