Class SqlWindowTableFunction

  • All Implemented Interfaces:
    org.apache.calcite.sql.SqlTableFunction
    Direct Known Subclasses:
    SqlCumulateTableFunction, SqlHopTableFunction, SqlSessionTableFunction, SqlTumbleTableFunction

    public class SqlWindowTableFunction
    extends org.apache.calcite.sql.SqlWindowTableFunction
    Base class for a table-valued function that computes windows. Examples include TUMBLE, HOP, CUMULATE and SESSION.

    Note: we extend Calcite's SqlWindowTableFunction, to support additional window_time time attribute column which should keep the same type with original time attribute.

    • Field Summary

      Fields 
      Modifier and Type Field Description
      static org.apache.calcite.sql.type.SqlReturnTypeInference ARG0_TABLE_FUNCTION_WINDOWING
      Type-inference strategy whereby the row type of a table function call is a ROW, which is combined from the row type of operand #0 (which is a TABLE) and two additional fields.
      protected static String GAP
      The gap interval, only used for SESSION window.
      protected static String PARAM_STEP
      The slide interval, only used for HOP window.
      • Fields inherited from class org.apache.calcite.sql.SqlWindowTableFunction

        PARAM_DATA, PARAM_KEY, PARAM_OFFSET, PARAM_SIZE, PARAM_SLIDE, PARAM_TIMECOL
      • Fields inherited from class org.apache.calcite.sql.SqlOperator

        kind, MDX_PRECEDENCE, NL
    • Constructor Summary

      Constructors 
      Constructor Description
      SqlWindowTableFunction​(String name, org.apache.calcite.sql.type.SqlOperandMetadata operandMetadata)
      Creates a window table function with a given name.
    • Method Summary

      All Methods Static Methods Instance Methods Concrete Methods 
      Modifier and Type Method Description
      boolean argumentMustBeScalar​(int ordinal)
      org.apache.calcite.sql.type.SqlOperandMetadata getOperandTypeChecker()  
      org.apache.calcite.sql.type.SqlReturnTypeInference getRowTypeInference()  
      static org.apache.calcite.rel.type.RelDataType inferRowType​(org.apache.calcite.rel.type.RelDataTypeFactory typeFactory, org.apache.calcite.rel.type.RelDataType inputRowType, org.apache.calcite.rel.type.RelDataType timeAttributeType)  
      void validateCall​(org.apache.calcite.sql.SqlCall call, org.apache.calcite.sql.validate.SqlValidator validator, org.apache.calcite.sql.validate.SqlValidatorScope scope, org.apache.calcite.sql.validate.SqlValidatorScope operandScope)  
      • Methods inherited from class org.apache.calcite.sql.SqlFunction

        deriveType, getFunctionType, getNameAsId, getParamNames, getParamTypes, getSqlIdentifier, getSyntax, isQuantifierAllowed, unparse, validateQuantifier
      • Methods inherited from class org.apache.calcite.sql.SqlOperator

        acceptCall, acceptCall, adjustType, allowsFraming, checkOperandCount, checkOperandTypes, constructArgNameList, constructArgTypeList, constructOperandList, createCall, createCall, createCall, createCall, createCall, createCall, createCall, equals, getAllowedSignatures, getAllowedSignatures, getKind, getLeftPrec, getMonotonicity, getMonotonicity, getName, getOperandCountRange, getOperandTypeInference, getReturnTypeInference, getRightPrec, getSignatureTemplate, getStrongPolicyInference, hashCode, inferReturnType, inferReturnType, isAggregator, isDeterministic, isDynamicFunction, isGroup, isGroupAuxiliary, isName, isSymmetrical, leftPrec, not, preValidateCall, requiresDecimalExpansion, requiresOrder, requiresOver, reverse, rewriteCall, rightPrec, toString, unparseListClause, unparseListClause, validateOperands, validRexOperands
      • Methods inherited from interface org.apache.calcite.sql.SqlTableFunction

        tableCharacteristic
    • Field Detail

      • ARG0_TABLE_FUNCTION_WINDOWING

        public static final org.apache.calcite.sql.type.SqlReturnTypeInference ARG0_TABLE_FUNCTION_WINDOWING
        Type-inference strategy whereby the row type of a table function call is a ROW, which is combined from the row type of operand #0 (which is a TABLE) and two additional fields. The fields are as follows:
        1. window_start: TIMESTAMP type to indicate a window's start
        2. window_end: TIMESTAMP type to indicate a window's end
        3. window_time: TIMESTAMP type with time attribute metadata to indicate a window's time attribute
    • Constructor Detail

      • SqlWindowTableFunction

        public SqlWindowTableFunction​(String name,
                                      org.apache.calcite.sql.type.SqlOperandMetadata operandMetadata)
        Creates a window table function with a given name.
    • Method Detail

      • getOperandTypeChecker

        public org.apache.calcite.sql.type.SqlOperandMetadata getOperandTypeChecker()
        Overrides:
        getOperandTypeChecker in class org.apache.calcite.sql.SqlWindowTableFunction
      • getRowTypeInference

        public org.apache.calcite.sql.type.SqlReturnTypeInference getRowTypeInference()
        Specified by:
        getRowTypeInference in interface org.apache.calcite.sql.SqlTableFunction
        Overrides:
        getRowTypeInference in class org.apache.calcite.sql.SqlWindowTableFunction
      • validateCall

        public void validateCall​(org.apache.calcite.sql.SqlCall call,
                                 org.apache.calcite.sql.validate.SqlValidator validator,
                                 org.apache.calcite.sql.validate.SqlValidatorScope scope,
                                 org.apache.calcite.sql.validate.SqlValidatorScope operandScope)
        Overrides:
        validateCall in class org.apache.calcite.sql.SqlFunction
      • argumentMustBeScalar

        public boolean argumentMustBeScalar​(int ordinal)

        Overrides because the first parameter of table-value function windowing is an explicit TABLE parameter, which is not scalar.

        Overrides:
        argumentMustBeScalar in class org.apache.calcite.sql.SqlWindowTableFunction
      • inferRowType

        public static org.apache.calcite.rel.type.RelDataType inferRowType​(org.apache.calcite.rel.type.RelDataTypeFactory typeFactory,
                                                                           org.apache.calcite.rel.type.RelDataType inputRowType,
                                                                           org.apache.calcite.rel.type.RelDataType timeAttributeType)