You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/06/12 08:45:25 UTC

[hive] branch master updated: HIVE-23269: Unsafe comparing bigints and strings (#992)

This is an automated email from the ASF dual-hosted git repository.

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new cd7252c  HIVE-23269: Unsafe comparing bigints and strings (#992)
cd7252c is described below

commit cd7252c7175c6f82731e619b16e3371565aaaec5
Author: dengzh <de...@gmail.com>
AuthorDate: Fri Jun 12 16:45:14 2020 +0800

    HIVE-23269: Unsafe comparing bigints and strings (#992)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |  2 +-
 .../hive/ql/parse/type/TypeCheckProcFactory.java   | 30 ++++++--
 .../ql/parse/type/TestBigIntCompareValidation.java | 79 ++++++++++++++++++++++
 3 files changed, 106 insertions(+), 5 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 24174ae..fce7fc3 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1663,7 +1663,7 @@ public class HiveConf extends Configuration {
         "Note that this check currently does not consider data size, only the query pattern."),
     HIVE_STRICT_CHECKS_TYPE_SAFETY("hive.strict.checks.type.safety", true,
         "Enabling strict type safety checks disallows the following:\n" +
-        "  Comparing bigints and strings.\n" +
+        "  Comparing bigints and strings/(var)chars.\n" +
         "  Comparing bigints and doubles."),
     HIVE_STRICT_CHECKS_CARTESIAN("hive.strict.checks.cartesian.product", false,
         "Enabling strict Cartesian join checks disallows the following:\n" +
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
index e16966e..f4a805c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/type/TypeCheckProcFactory.java
@@ -28,8 +28,10 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Stack;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -776,6 +778,20 @@ public class TypeCheckProcFactory<T> {
       return getDefaultExprProcessor().getFuncExprNodeDescWithUdfData(baseType, tableFieldTypeInfo, column);
     }
 
+    private boolean unSafeCompareWithBigInt(TypeInfo otherTypeInfo, TypeInfo bigintCandidate) {
+      Set<PrimitiveObjectInspector.PrimitiveCategory> unsafeConventionTyps = Sets.newHashSet(
+          PrimitiveObjectInspector.PrimitiveCategory.STRING,
+          PrimitiveObjectInspector.PrimitiveCategory.VARCHAR,
+          PrimitiveObjectInspector.PrimitiveCategory.CHAR);
+
+      if (bigintCandidate.equals(TypeInfoFactory.longTypeInfo) && otherTypeInfo instanceof PrimitiveTypeInfo) {
+        PrimitiveObjectInspector.PrimitiveCategory pCategory =
+            ((PrimitiveTypeInfo)otherTypeInfo).getPrimitiveCategory();
+        return unsafeConventionTyps.contains(pCategory);
+      }
+      return false;
+    }
+
     protected void validateUDF(ASTNode expr, boolean isFunction, TypeCheckCtx ctx, FunctionInfo fi,
         List<T> children, GenericUDF genericUDF) throws SemanticException {
       // Check if a bigint is implicitely cast to a double as part of a comparison
@@ -790,11 +806,17 @@ public class TypeCheckProcFactory<T> {
         LogHelper console = new LogHelper(LOG);
 
         // For now, if a bigint is going to be cast to a double throw an error or warning
-        if ((oiTypeInfo0.equals(TypeInfoFactory.stringTypeInfo) && oiTypeInfo1.equals(TypeInfoFactory.longTypeInfo)) ||
-            (oiTypeInfo0.equals(TypeInfoFactory.longTypeInfo) && oiTypeInfo1.equals(TypeInfoFactory.stringTypeInfo))) {
+        if (unSafeCompareWithBigInt(oiTypeInfo0, oiTypeInfo1) || unSafeCompareWithBigInt(oiTypeInfo1, oiTypeInfo0)) {
           String error = StrictChecks.checkTypeSafety(conf);
-          if (error != null) throw new UDFArgumentException(error);
-          console.printError("WARNING: Comparing a bigint and a string may result in a loss of precision.");
+          if (error != null) {
+            throw new UDFArgumentException(error);
+          }
+          // To  make the error output be consistency, get the other side type name that comparing with biginit.
+          String type = oiTypeInfo0.getTypeName();
+          if (!oiTypeInfo1.equals(TypeInfoFactory.longTypeInfo)) {
+            type = oiTypeInfo1.getTypeName();
+          }
+          console.printError("WARNING: Comparing a bigint and a " + type + " may result in a loss of precision.");
         } else if ((oiTypeInfo0.equals(TypeInfoFactory.doubleTypeInfo) && oiTypeInfo1.equals(TypeInfoFactory.longTypeInfo)) ||
             (oiTypeInfo0.equals(TypeInfoFactory.longTypeInfo) && oiTypeInfo1.equals(TypeInfoFactory.doubleTypeInfo))) {
           console.printError("WARNING: Comparing a bigint and a double may result in a loss of precision.");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestBigIntCompareValidation.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestBigIntCompareValidation.java
new file mode 100644
index 0000000..5ed9b4b
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/type/TestBigIntCompareValidation.java
@@ -0,0 +1,79 @@
+/*
+ * 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.hadoop.hive.ql.parse.type;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test that a bigint comparing with a string, varchar or char is not allowed by default.
+ *
+ */
+public class TestBigIntCompareValidation {
+
+  private ExprNodeConstantDesc constant;
+  private TypeCheckProcFactory.DefaultExprProcessor processor;
+  private String errorMsg;
+  private FunctionInfo functionInfo;
+
+  @Before
+  public void setUp() throws Exception {
+    this.constant = new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, 0L);
+    this.processor = ExprNodeTypeCheck.getExprNodeDefaultExprProcessor();
+    this.errorMsg = HiveConf.StrictChecks.checkTypeSafety(new HiveConf());
+    this.functionInfo = FunctionRegistry.getFunctionInfo("=");
+  }
+
+  @Test
+  public void testCompareWithVarchar() {
+    ExprNodeDesc nodeDesc = new ExprNodeColumnDesc(TypeInfoFactory.varcharTypeInfo, "_c0", null, false);
+    testValidateUDFOnComparingBigInt(nodeDesc);
+  }
+
+  @Test
+  public void testCompareWithString() {
+    ExprNodeDesc nodeDesc = new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, "_c1", null, false);
+    testValidateUDFOnComparingBigInt(nodeDesc);
+  }
+
+  @Test
+  public void testCompareWithChar() {
+    ExprNodeDesc nodeDesc = new ExprNodeColumnDesc(TypeInfoFactory.charTypeInfo, "_c2", null, false);
+    testValidateUDFOnComparingBigInt(nodeDesc);
+  }
+
+  private void testValidateUDFOnComparingBigInt(ExprNodeDesc nodeDesc) {
+    try {
+      TypeCheckCtx ctx = new TypeCheckCtx(null);
+      processor.validateUDF(null, false, ctx, functionInfo,
+          Lists.newArrayList(constant, nodeDesc), functionInfo.getGenericUDF());
+      Assert.fail("Should throw exception as comparing a bigint and a " + nodeDesc.getTypeString());
+    } catch (Exception e) {
+      Assert.assertEquals(errorMsg, e.getMessage());
+    }
+  }
+}