You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "tjbanghart (via GitHub)" <gi...@apache.org> on 2023/06/05 19:23:05 UTC

[GitHub] [calcite] tjbanghart commented on a diff in pull request #3245: [CALCITE-5747] Conflicting FLOOR return type between Calcite and BigQ…

tjbanghart commented on code in PR #3245:
URL: https://github.com/apache/calcite/pull/3245#discussion_r1218492397


##########
core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java:
##########
@@ -149,6 +149,16 @@ public BigQuerySqlDialect(SqlDialect.Context context) {
   @Override public void unparseCall(final SqlWriter writer, final SqlCall call, final int leftPrec,
       final int rightPrec) {
     switch (call.getKind()) {
+    case CEIL_BIG_QUERY:
+      final SqlWriter.Frame ceilFrame = writer.startFunCall("CEIL");
+      call.operand(0).unparse(writer, leftPrec, rightPrec);
+      writer.endFunCall(ceilFrame);
+      break;
+    case FLOOR_BIG_QUERY:

Review Comment:
   Can the unparse logic be pushed to the operator itself? Otherwise I think each dialect will have to override the `BIG_QUERY` style function calls (e.g. BQ incoming query targeting a Postgres backend).



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java:
##########
@@ -41,22 +45,23 @@
 public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlFloorFunction(SqlKind kind) {
-    super(kind.name(), kind, ReturnTypes.ARG0_OR_EXACT_NO_SCALE, null,
+  public SqlFloorFunction(SqlKind kind, SqlReturnTypeInference returnTypeInference) {
+    super(kind.name(), kind, returnTypeInference, null,
         OperandTypes.NUMERIC_OR_INTERVAL.or(
             OperandTypes.sequence("'" + kind + "(<DATE> TO <TIME_UNIT>)'\n"
                     + "'" + kind + "(<TIME> TO <TIME_UNIT>)'\n"
                     + "'" + kind + "(<TIMESTAMP> TO <TIME_UNIT>)'",
                 OperandTypes.DATETIME,
                 OperandTypes.ANY)),
         SqlFunctionCategory.NUMERIC);
-    Preconditions.checkArgument(kind == SqlKind.FLOOR || kind == SqlKind.CEIL);
+    Preconditions.checkArgument(kind == SqlKind.FLOOR || kind == SqlKind.CEIL
+        || kind == SqlKind.FLOOR_BIG_QUERY || kind == SqlKind.CEIL_BIG_QUERY);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
-    // Monotonic iff its first argument is, but not strict.
+    // Monotonic if its first argument is, but not strict.
     return call.getOperandMonotonicity(0).unstrict();
   }
 

Review Comment:
   Consider moving the unparsing logic for the BQ style functions here.



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java:
##########
@@ -41,22 +45,23 @@
 public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
   //~ Constructors -----------------------------------------------------------
 
-  public SqlFloorFunction(SqlKind kind) {
-    super(kind.name(), kind, ReturnTypes.ARG0_OR_EXACT_NO_SCALE, null,
+  public SqlFloorFunction(SqlKind kind, SqlReturnTypeInference returnTypeInference) {
+    super(kind.name(), kind, returnTypeInference, null,
         OperandTypes.NUMERIC_OR_INTERVAL.or(
             OperandTypes.sequence("'" + kind + "(<DATE> TO <TIME_UNIT>)'\n"
                     + "'" + kind + "(<TIME> TO <TIME_UNIT>)'\n"
                     + "'" + kind + "(<TIMESTAMP> TO <TIME_UNIT>)'",
                 OperandTypes.DATETIME,
                 OperandTypes.ANY)),
         SqlFunctionCategory.NUMERIC);
-    Preconditions.checkArgument(kind == SqlKind.FLOOR || kind == SqlKind.CEIL);
+    Preconditions.checkArgument(kind == SqlKind.FLOOR || kind == SqlKind.CEIL
+        || kind == SqlKind.FLOOR_BIG_QUERY || kind == SqlKind.CEIL_BIG_QUERY);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
-    // Monotonic iff its first argument is, but not strict.
+    // Monotonic if its first argument is, but not strict.

Review Comment:
   Might be intentional for "if and only if"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org