def getExpressionString()

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")
    }
  }