|
55 | 55 | import org.elasticsearch.xpack.esql.expression.predicate.logical.Not; |
56 | 56 | import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; |
57 | 57 | import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison; |
| 58 | +import org.elasticsearch.xpack.esql.inference.InferenceCommandConfig; |
58 | 59 | import org.elasticsearch.xpack.esql.parser.promql.PromqlParserUtils; |
59 | 60 | import org.elasticsearch.xpack.esql.plan.EsqlStatement; |
60 | 61 | import org.elasticsearch.xpack.esql.plan.IndexPattern; |
@@ -1119,9 +1120,14 @@ public PlanFactory visitRerankCommand(EsqlBaseParser.RerankCommandContext ctx) { |
1119 | 1120 | throw qualifiersUnsupportedInFieldDefinitions(scoreAttribute.source(), ctx.targetField.getText()); |
1120 | 1121 | } |
1121 | 1122 |
|
1122 | | - Literal rowLimit = Literal.integer(Source.EMPTY, context.inferenceCommandConfigProvider().get("rerank").rowLimit()); |
| 1123 | + InferenceCommandConfig commandConfig = context.inferenceCommandConfigProvider().get("rerank"); |
| 1124 | + Literal rowLimit = Literal.integer(Source.EMPTY, commandConfig.rowLimit()); |
1123 | 1125 |
|
1124 | 1126 | return p -> { |
| 1127 | + if (commandConfig.enabled() == false) { |
| 1128 | + throw new ParsingException(source, "RERANK command is disabled"); |
| 1129 | + } |
| 1130 | + |
1125 | 1131 | checkForRemoteClusters(p, source, "RERANK"); |
1126 | 1132 | return applyRerankOptions( |
1127 | 1133 | new Rerank(source, p, queryText, rerankFields, scoreAttribute, rowLimit), |
@@ -1164,11 +1170,17 @@ public PlanFactory visitCompletionCommand(EsqlBaseParser.CompletionCommandContex |
1164 | 1170 | if (targetField.qualifier() != null) { |
1165 | 1171 | throw qualifiersUnsupportedInFieldDefinitions(targetField.source(), ctx.targetField.getText()); |
1166 | 1172 | } |
1167 | | - |
1168 | | - Literal rowLimit = Literal.integer(Source.EMPTY, context.inferenceCommandConfigProvider().get("completion").rowLimit()); |
| 1173 | + |
| 1174 | + InferenceCommandConfig commandConfig = context.inferenceCommandConfigProvider().get("completion"); |
| 1175 | + Literal rowLimit = Literal.integer(Source.EMPTY, commandConfig.rowLimit()); |
1169 | 1176 |
|
1170 | 1177 | return p -> { |
| 1178 | + if (commandConfig.enabled() == false) { |
| 1179 | + throw new ParsingException(source, "COMPLETION command is disabled"); |
| 1180 | + } |
| 1181 | + |
1171 | 1182 | checkForRemoteClusters(p, source, "COMPLETION"); |
| 1183 | + |
1172 | 1184 | return applyCompletionOptions(new Completion(source, p, prompt, targetField, rowLimit), ctx.commandNamedParameters()); |
1173 | 1185 | }; |
1174 | 1186 | } |
|
0 commit comments