1616 */
1717package org .apache .calcite .sql .validate ;
1818
19+ import org .apache .flink .table .api .ValidationException ;
1920import org .apache .flink .table .planner .calcite .FlinkSqlCallBinding ;
2021import org .apache .flink .table .planner .functions .sql .ml .SqlVectorSearchTableFunction ;
2122
178179 *
179180 * <p>Lines 2571 ~ 2588, CALCITE-7217, should be removed after upgrading Calcite to 1.41.0.
180181 *
181- * <p>Lines 3840 ~ 3844, 6511 ~ 6517 Flink improves Optimize the retrieval of sub-operands in
182+ * <p>Line 2618 ~2631, set the correct scope for VECTOR_SEARCH.
183+ *
184+ * <p>Lines 3920 ~ 3925, 6599 ~ 6606 Flink improves Optimize the retrieval of sub-operands in
182185 * SqlCall when using NamedParameters at {@link SqlValidatorImpl#checkRollUp}.
183186 *
184- * <p>Lines 5315 ~ 5321 , FLINK-24352 Add null check for temporal table check on SqlSnapshot.
187+ * <p>Lines 5340 ~ 5347 , FLINK-24352 Add null check for temporal table check on SqlSnapshot.
185188 */
186189public class SqlValidatorImpl implements SqlValidatorWithHints {
187190 // ~ Static fields/initializers ---------------------------------------------
@@ -2571,6 +2574,10 @@ private SqlNode registerFrom(
25712574 case LATERAL :
25722575 // ----- FLINK MODIFICATION BEGIN -----
25732576 SqlBasicCall sbc = (SqlBasicCall ) node ;
2577+ // Put the usingScope which is a JoinScope,
2578+ // in order to make visible the left items
2579+ // of the JOIN tree.
2580+ scopes .put (node , usingScope );
25742581 registerFrom (
25752582 parentScope ,
25762583 usingScope ,
@@ -2581,10 +2588,6 @@ private SqlNode registerFrom(
25812588 extendList ,
25822589 forceNullable ,
25832590 true );
2584- // Put the usingScope which is a JoinScope,
2585- // in order to make visible the left items
2586- // of the JOIN tree.
2587- scopes .put (node , usingScope );
25882591 return sbc ;
25892592 // ----- FLINK MODIFICATION END -----
25902593
@@ -2625,8 +2628,14 @@ private SqlNode registerFrom(
26252628 .isA (
26262629 new HashSet <>(
26272630 Collections .singletonList (SqlKind .SELECT )))) {
2631+ boolean queryColumnIsNotLiteral =
2632+ binding .operand (2 ).getKind () != SqlKind .LITERAL ;
2633+ if (!queryColumnIsNotLiteral && !lateral ) {
2634+ throw new ValidationException (
2635+ "The query column is not literal, please use LATERAL TABLE to run VECTOR_SEARCH." );
2636+ }
26282637 SqlValidatorScope scope = getSelectScope ((SqlSelect ) binding .operand (0 ));
2629- scopes .put (node , scope );
2638+ scopes .put (enclosingNode , scope );
26302639 return newNode ;
26312640 }
26322641 // ----- FLINK MODIFICATION END -----
0 commit comments