in flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala [462:572]
def getExpressionString(
expr: RexNode,
inFields: Seq[String],
localExprsTable: Option[List[RexNode]],
expressionFormat: ExpressionFormat,
expressionDetail: ExpressionDetail): String = {
expr match {
case pr: RexPatternFieldRef =>
val alpha = pr.getAlpha
val field = inFields(pr.getIndex)
s"$alpha.$field"
case i: RexInputRef =>
inFields(i.getIndex)
case l: RexLiteral =>
expressionDetail match {
case ExpressionDetail.Digest =>
// the digest for the literal
l.toString
case ExpressionDetail.Explain =>
val value = l.getValue
if (value == null) {
// return null with type
"null:" + l.getType
} else {
l.getTypeName match {
case SqlTypeName.DOUBLE => Util.toScientificNotation(value.asInstanceOf[BigDecimal])
case SqlTypeName.BIGINT => s"${value.asInstanceOf[BigDecimal].longValue()}L"
case SqlTypeName.BINARY => s"X'${value.asInstanceOf[ByteString].toString(16)}'"
case SqlTypeName.VARCHAR | SqlTypeName.CHAR =>
s"'${value.asInstanceOf[NlsString].getValue}'"
case SqlTypeName.TIME | SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE =>
l.getValueAs(classOf[TimeString]).toString
case SqlTypeName.DATE =>
l.getValueAs(classOf[DateString]).toString
case SqlTypeName.TIMESTAMP | SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE =>
l.getValueAs(classOf[TimestampString]).toString
case typ if SqlTypeName.INTERVAL_TYPES.contains(typ) => l.toString
case _ => value.toString
}
}
}
case _: RexLocalRef if localExprsTable.isEmpty =>
throw new IllegalArgumentException(
"Encountered RexLocalRef without " +
"local expression table")
case l: RexLocalRef =>
val lExpr = localExprsTable.get(l.getIndex)
getExpressionString(lExpr, inFields, localExprsTable, expressionFormat, expressionDetail)
case c: RexCall =>
val op = c.getOperator.toString
val ops = c.getOperands.map(
getExpressionString(_, inFields, localExprsTable, expressionFormat, expressionDetail))
c.getOperator match {
case _: SqlAsOperator => ops.head
case _: SqlCastFunction | _: SqlTryCastFunction =>
val typeStr = expressionDetail match {
case ExpressionDetail.Digest => c.getType.getFullTypeString
case ExpressionDetail.Explain => c.getType.toString
}
s"$op(${ops.head} AS $typeStr)"
case _ =>
if (ops.size() == 1) {
val operand = ops.head
expressionFormat match {
case ExpressionFormat.Infix =>
c.getKind match {
case SqlKind.IS_FALSE | SqlKind.IS_NOT_FALSE | SqlKind.IS_TRUE |
SqlKind.IS_NOT_TRUE | SqlKind.IS_UNKNOWN | SqlKind.IS_NULL |
SqlKind.IS_NOT_NULL =>
s"$operand $op"
case _ => s"$op($operand)"
}
case ExpressionFormat.PostFix => s"$operand $op"
case ExpressionFormat.Prefix => s"$op($operand)"
}
} else {
c.getKind match {
case SqlKind.TIMES | SqlKind.DIVIDE | SqlKind.PLUS | SqlKind.MINUS |
SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL | SqlKind.GREATER_THAN |
SqlKind.GREATER_THAN_OR_EQUAL | SqlKind.EQUALS | SqlKind.NOT_EQUALS |
SqlKind.OR | SqlKind.AND =>
expressionFormat match {
case ExpressionFormat.Infix => s"(${ops.mkString(s" $op ")})"
case ExpressionFormat.PostFix => s"(${ops.mkString(", ")})$op"
case ExpressionFormat.Prefix => s"$op(${ops.mkString(", ")})"
}
case _ => s"$op(${ops.mkString(", ")})"
}
}
}
case fa: RexFieldAccess =>
val referenceExpr = getExpressionString(
fa.getReferenceExpr,
inFields,
localExprsTable,
expressionFormat,
expressionDetail)
val field = fa.getField.getName
s"$referenceExpr.$field"
case cv: RexCorrelVariable =>
cv.toString
case _ =>
throw new IllegalArgumentException(s"Unknown expression type '${expr.getClass}': $expr")
}
}