You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "gortiz (via GitHub)" <gi...@apache.org> on 2024/02/22 16:15:09 UTC

[PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

gortiz opened a new pull request, #12475:
URL: https://github.com/apache/pinot/pull/12475

   This PR adds a check that fails if there is an implicit or explicit cast from to Bytes.
   
   I still need to add some automatic tests, but I've tested manually. In a query like:
   ```sql
   select * from starbucksStores where location_st_point = 
   '80c062bc98021f94f1404e9bda0f6b0202'
   limit 10
   ```
   
   The message shown is: ` From line 0, column 0 to line 2, column 36: Cannot cast '80c062bc98021f94f1404e9bda0f6b0202' as VARBINARY. Try to use binary literal instead (like x'80c062bc98021f94f1404e9bda0f6b0202')`
   
   In a query like:
   ```sql
   select * from starbucksStores 
   where OCTET_LENGTH(
     substring('80c062bc98021f94f1404e9bda0f6b0202', 2)
   ) > 0 limit 10
   ```
   
   The message shown is: `From line 0, column 0 to line 3, column 52: Cannot cast SUBSTRING('80c062bc98021f94f1404e9bda0f6b0202' FROM 2) as VARBINARY. Try to wrap the expression in hexToBytes (like hexToBytes(SUBSTRING('80c062bc98021f94f1404e9bda0f6b0202' FROM 2)))`
   
   Two things that we can try to improve (although priority doesn't seem high):
   - Given in this cases the cast expression was implicit, start position seem to be line 0 column 0. It would be great to find a way to show a more precise start position
   - As shown in the second case, the suggested expression is not exactly like the original one.
   
   Related to https://github.com/apache/pinot/issues/12457


-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1508312558


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")

Review Comment:
   ^^ Should we remove this?



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1507181049


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);
+      }
+      SqlNode srcNode = operands.get(0);
+      RelDataType fromType = _originalValidator.getValidatedNodeTypeIfKnown(srcNode);
+      if (fromType != null && SqlTypeUtil.isBinary(fromType)) {

Review Comment:
   In Calcite there is BINARY and VARBINARY. In both cases `SqlTypeUtil.isBinary` return true. I'm assuming we want to allow castings between them.



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1508720811


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")

Review Comment:
   Removed



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1507187948


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);

Review Comment:
   It is not the same, right?
   
   `return null` just aborts the execution here while `return super.visit(call)` does in fact return the same value, but has the desired side effect of visiting the operators.
   
   For example, an expression like: `CAST (someOperation(CAST ('someliteral' as BINARY)) as STRING)` will be converted into:
   1. CAST
      1. someOperation
          1.  CAST
              1. 'someliteral'
              2. BINARY
      2. STRING
   
   In case we changed this line to `return null` we won't be able to detect the inner cast to BINARY because we would just abort the visitor once the first CAST to STRING was detected.



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "gortiz (via GitHub)" <gi...@apache.org>.
gortiz commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1507194978


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);
+      }
+      SqlNode srcNode = operands.get(0);
+      RelDataType fromType = _originalValidator.getValidatedNodeTypeIfKnown(srcNode);
+      if (fromType != null && SqlTypeUtil.isBinary(fromType)) {
+        return super.visit(call);
+      }
+      String message = "Cannot cast " + srcNode + " as " + toType + ".";
+      if (srcNode instanceof SqlCharStringLiteral) {
+        message += " Try to use binary literal instead (like x" + srcNode + ")";
+      } else if (fromType != null && SqlTypeUtil.isCharacter(fromType)) {
+        message += " Try to wrap the expression in hexToBytes (like hexToBytes(" + srcNode + "))";
+      }
+      SqlParserPos pos = call.getParserPosition();
+      RuntimeException ex = new RuntimeException(message);

Review Comment:
   Changed to throw a new runtime exception called `InvalidCastException`



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1506782839


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")

Review Comment:
   (minor) which warning do we violate here?



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);
+      }
+      SqlNode srcNode = operands.get(0);
+      RelDataType fromType = _originalValidator.getValidatedNodeTypeIfKnown(srcNode);
+      if (fromType != null && SqlTypeUtil.isBinary(fromType)) {
+        return super.visit(call);
+      }
+      String message = "Cannot cast " + srcNode + " as " + toType + ".";
+      if (srcNode instanceof SqlCharStringLiteral) {
+        message += " Try to use binary literal instead (like x" + srcNode + ")";

Review Comment:
   (minor) Suggest following calcite [documentation](https://calcite.apache.org/javadocAggregate/org/apache/calcite/sql/SqlLiteral.html) which uses capital case `X`



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);
+      }
+      SqlNode srcNode = operands.get(0);
+      RelDataType fromType = _originalValidator.getValidatedNodeTypeIfKnown(srcNode);
+      if (fromType != null && SqlTypeUtil.isBinary(fromType)) {

Review Comment:
   Will we every cast from binary to binary?



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);

Review Comment:
   Same for other short circuit call
   ```suggestion
           return null;
   ```



##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);
+      }
+      SqlNode srcNode = operands.get(0);
+      RelDataType fromType = _originalValidator.getValidatedNodeTypeIfKnown(srcNode);
+      if (fromType != null && SqlTypeUtil.isBinary(fromType)) {
+        return super.visit(call);
+      }
+      String message = "Cannot cast " + srcNode + " as " + toType + ".";
+      if (srcNode instanceof SqlCharStringLiteral) {
+        message += " Try to use binary literal instead (like x" + srcNode + ")";
+      } else if (fromType != null && SqlTypeUtil.isCharacter(fromType)) {
+        message += " Try to wrap the expression in hexToBytes (like hexToBytes(" + srcNode + "))";
+      }
+      SqlParserPos pos = call.getParserPosition();
+      RuntimeException ex = new RuntimeException(message);

Review Comment:
   Should we throw a more specific exception?



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12475:
URL: https://github.com/apache/pinot/pull/12475#issuecomment-1968677570

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: Patch coverage is `44.00000%` with `14 lines` in your changes are missing coverage. Please review.
   > Project coverage is 61.58%. Comparing base [(`59551e4`)](https://app.codecov.io/gh/apache/pinot/commit/59551e45224f1535c4863fd577622b37366ccc97?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) to head [(`6a70ffa`)](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   > Report is 21 commits behind head on master.
   
   | [Files](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [.../apache/pinot/query/validate/BytesCastVisitor.java](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvdmFsaWRhdGUvQnl0ZXNDYXN0VmlzaXRvci5qYXZh) | 41.66% | [13 Missing and 1 partial :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #12475      +/-   ##
   ============================================
   - Coverage     61.75%   61.58%   -0.17%     
     Complexity      207      207              
   ============================================
     Files          2436     2450      +14     
     Lines        133233   133528     +295     
     Branches      20636    20690      +54     
   ============================================
   - Hits          82274    82233      -41     
   - Misses        44911    45214     +303     
   - Partials       6048     6081      +33     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (ø)` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.67% <0.00%> (-34.04%)` | :arrow_down: |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.88% <44.00%> (-26.74%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `61.54% <44.00%> (-0.20%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `34.86% <44.00%> (+7.14%)` | :arrow_up: |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `61.58% <44.00%> (-0.17%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `61.58% <44.00%> (-0.17%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.70% <44.00%> (-0.19%)` | :arrow_down: |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12475/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.67% <0.00%> (-0.06%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12475?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12475:
URL: https://github.com/apache/pinot/pull/12475#discussion_r1508311185


##########
pinot-query-planner/src/main/java/org/apache/pinot/query/validate/BytesCastVisitor.java:
##########
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.query.validate;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlCastFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Static;
+
+
+public class BytesCastVisitor extends SqlBasicVisitor<Void> {
+
+  private final SqlValidator _originalValidator;
+
+  public BytesCastVisitor(SqlValidator originalValidator) {
+    _originalValidator = originalValidator;
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  public Void visit(SqlCall call) {
+    if (call.getOperator() instanceof SqlCastFunction) {
+      List<SqlNode> operands = call.getOperandList();
+      SqlNode sqlNode = operands.get(1);
+      Preconditions.checkState(sqlNode instanceof SqlDataTypeSpec);
+      RelDataType toType = ((SqlDataTypeSpec) sqlNode).deriveType(_originalValidator);
+      if (!SqlTypeUtil.isBinary(toType)) {
+        return super.visit(call);

Review Comment:
   I see, make sense!



-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


Re: [PR] Add a post-validator visitor that verifies there are no cast to bytes [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #12475:
URL: https://github.com/apache/pinot/pull/12475


-- 
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@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org