You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by db...@apache.org on 2023/07/11 16:06:40 UTC

[impala] branch master updated: IMPALA-10753: Incorrect length when multiple CHAR(N) values are inserted

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 31137cc0e IMPALA-10753: Incorrect length when multiple CHAR(N) values are inserted
31137cc0e is described below

commit 31137cc0e373fa2e5f9f6823c3763d5c03d9f93a
Author: Daniel Becker <da...@cloudera.com>
AuthorDate: Mon Aug 29 14:34:38 2022 +0200

    IMPALA-10753: Incorrect length when multiple CHAR(N) values are inserted
    
    If, in a VALUES clause, for the same column all of the values are CHAR
    types but not all are of the same length, the common type chosen is
    CHAR(max(lengths)). This means that shorter values are padded with
    spaces. If the destination column is not CHAR but VARCHAR or STRING,
    this produces different results than if the values in the column are
    inserted individually, in separate statements. This behaviour is
    suboptimal because information is lost.
    
    For example:
      CREATE TABLE impala_char_insert (s STRING);
    
      -- all values are CHAR(N) with different N, but all will use the
         biggest N
      INSERT OVERWRITE impala_char_insert VALUES
        (CAST("1" AS CHAR(1))),
        (CAST("12" AS CHAR(2))),
        (CAST("123" AS CHAR(3)));
    
      SELECT length(s) FROM impala_char_insert;
      3
      3
      3
    
      -- if inserted individually, the result is
      SELECT length(s) FROM impala_char_insert;
      1
      2
      3
    
    This patch adds the query option VALUES_STMT_AVOID_LOSSY_CHAR_PADDING
    which, when set to true, fixes the problem by implicitly casting the
    values to the VARCHAR type of the longest value if all values in a
    column are CHAR types AND not all have the same length. This VARCHAR
    type will be the common type of the column in the VALUES statement.
    
    The new behaviour is not turned on by default because it is a breaking
    change.
    
    Note that the behaviour in Hive is different from both behaviours in
    Impala: Hive (and PostgreSQL) implicitly remove trailing spaces from
    CHAR values when they are cast to other types, which is also lossy.
    
    We choose VARCHAR instead of STRING as the common type because VARCHAR
    can be converted to any VARCHAR type shorter or the same length and also
    to STRING, while STRING cannot safely be converted to VARCHAR because
    its length is not bounded - we would therefore run into problems if the
    common type were STRING and the destination column were VARCHAR.
    
    Note: although the VALUES statement is implemented as a special UNION
    operation under the hood, this patch doesn't change the behaviour of
    explicit UNION statements, it only applies to VALUES statements.
    
    Note: the new VALUES_STMT_AVOID_LOSSY_CHAR_PADDING query option and
    ALLOW_UNSAFE_CASTS are not allowed to be used at the same time: if both
    are set to true and the query contains set operation(s), an error is
    returned.
    
    Testing:
     - Added tests verifying that unneeded padding doesn't occur and the
       queries succeed in various situations, e.g. different destination
       column types and multi-column inserts. See
       testdata/workloads/functional-query/queries/QueryTest/chars-values-clause.test
    
    Change-Id: I9e9e189cb3c2be0e741ca3d15a7f97ec3a1b1a86
    Reviewed-on: http://gerrit.cloudera.org:8080/18999
    Reviewed-by: Csaba Ringhofer <cs...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/query-options.cc                    |   5 +
 be/src/service/query-options.h                     |   7 +-
 common/thrift/ImpalaService.thrift                 |   6 +
 common/thrift/Query.thrift                         |   3 +
 .../java/org/apache/impala/analysis/Analyzer.java  |  35 ++++-
 .../apache/impala/analysis/SetOperationStmt.java   |  25 ++-
 .../org/apache/impala/analysis/StatementBase.java  |  30 +++-
 .../org/apache/impala/analysis/ValuesStmt.java     |   6 +
 .../main/java/org/apache/impala/catalog/Type.java  |   1 +
 .../org/apache/impala/planner/PlanFragment.java    |   2 +-
 .../chars-values-stmt-lossy-char-padding.test      |  33 ++++
 .../chars-values-stmt-no-lossy-char-padding.test   | 173 +++++++++++++++++++++
 tests/query_test/test_chars.py                     |  15 ++
 13 files changed, 331 insertions(+), 10 deletions(-)

diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index 7df214152..c01ffc654 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1106,6 +1106,7 @@ Status impala::SetQueryOption(const string& key, const string& value,
         RETURN_IF_ERROR(
             QueryOptionValidator<int32_t>::NotEquals(option, int32_t_val, 1));
         query_options->__set_max_sort_run_size(int32_t_val);
+        break;
       }
       case TImpalaQueryOptions::ALLOW_UNSAFE_CASTS: {
         query_options->__set_allow_unsafe_casts(IsTrue(value));
@@ -1122,6 +1123,10 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_disable_optimized_iceberg_v2_read(IsTrue(value));
         break;
       }
+      case TImpalaQueryOptions::VALUES_STMT_AVOID_LOSSY_CHAR_PADDING: {
+        query_options->__set_values_stmt_avoid_lossy_char_padding(IsTrue(value));
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index 2a585c243..79b8e6646 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -50,7 +50,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // time we add or remove a query option to/from the enum TImpalaQueryOptions.
 #define QUERY_OPTS_TABLE                                                                 \
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),                                 \
-      TImpalaQueryOptions::DISABLE_OPTIMIZED_ICEBERG_V2_READ + 1);                       \
+      TImpalaQueryOptions::VALUES_STMT_AVOID_LOSSY_CHAR_PADDING + 1);                    \
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED) \
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)               \
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)             \
@@ -297,7 +297,10 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
   QUERY_OPT_FN(num_threads_for_table_migration, NUM_THREADS_FOR_TABLE_MIGRATION,         \
       TQueryOptionLevel::ADVANCED)                                                       \
   QUERY_OPT_FN(disable_optimized_iceberg_v2_read, DISABLE_OPTIMIZED_ICEBERG_V2_READ,     \
-      TQueryOptionLevel::ADVANCED);
+      TQueryOptionLevel::ADVANCED)                                                       \
+  QUERY_OPT_FN(values_stmt_avoid_lossy_char_padding,                                     \
+      VALUES_STMT_AVOID_LOSSY_CHAR_PADDING, TQueryOptionLevel::REGULAR)                  \
+  ;
 
 /// Enforce practical limits on some query options to avoid undesired query state.
 static const int64_t SPILLABLE_BUFFER_LIMIT = 1LL << 40; // 1 TB
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index 2b82825c3..49862e892 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -806,6 +806,12 @@ enum TImpalaQueryOptions {
 
   // Turns off optimized Iceberg V2 reads, falls back to Hash Join
   DISABLE_OPTIMIZED_ICEBERG_V2_READ = 160;
+
+  // In VALUES clauses, if all values in a column are CHARs but they have different
+  // lengths, choose the VARCHAR type of the longest length instead of the corresponding
+  // CHAR type as the common type. This avoids padding and thereby loss of information.
+  // See IMPALA-10753.
+  VALUES_STMT_AVOID_LOSSY_CHAR_PADDING = 161;
 }
 
 // The summary of a DML statement.
diff --git a/common/thrift/Query.thrift b/common/thrift/Query.thrift
index 2e866cd6c..6a610f4b0 100644
--- a/common/thrift/Query.thrift
+++ b/common/thrift/Query.thrift
@@ -646,6 +646,9 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift
   161: optional bool disable_optimized_iceberg_v2_read = false;
+
+  // See comment in ImpalaService.thrift
+  162: optional bool values_stmt_avoid_lossy_char_padding = false;
 }
 
 // Impala currently has three types of sessions: Beeswax, HiveServer2 and external
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index a5bba3f27..d8cfa2cf6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -63,6 +63,7 @@ import org.apache.impala.catalog.FeView;
 import org.apache.impala.catalog.IcebergTimeTravelTable;
 import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.MaterializedViewHdfsTable;
+import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.StructField;
 import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.TableLoadingException;
@@ -3374,9 +3375,13 @@ public class Analyzer {
    * widest compatible expression encountered among all i-th exprs in the expr lists.
    * Returns null if an empty expression list or null is passed to it.
    * Throw an AnalysisException if the types are incompatible.
+   *
+   * If 'avoidLossyCharPadding' is true, then if a column contains only CHAR values but
+   * the lengths are different, they are cast to VARCHAR so the common type will not be a
+   * CHAR type, which would cause padding. See IMPALA-10753.
    */
-  public List<Expr> castToSetOpCompatibleTypes(List<List<Expr>> exprLists)
-      throws AnalysisException {
+  public List<Expr> castToSetOpCompatibleTypes(List<List<Expr>> exprLists,
+      boolean avoidLossyCharPadding) throws AnalysisException {
     if (exprLists == null || exprLists.size() == 0) return null;
     if (exprLists.size() == 1) return exprLists.get(0);
 
@@ -3418,7 +3423,24 @@ public class Analyzer {
         }
 
         // compatibleType will be updated if a new wider type is encountered
-        if (preType != compatibleType) widestExprs.set(i, expr);
+        if (preType != compatibleType) {
+          if (avoidLossyCharPadding && differentLenCharTypes(preType, compatibleType)) {
+            // ALLOW_UNSAFE_CASTS and VALUES_STMT_AVOID_LOSSY_CHAR_PADDING don't work well
+            // together now. With ALLOW_UNSAFE_CASTS it is possible that 'compatibleType'
+            // alternates between integer and CHAR types, so we don't detect that we
+            // should change to VARCHAR (when VALUES_STMT_AVOID_LOSSY_CHAR_PADDING is
+            // true). Having these two query options at the same time is disabled in
+            // 'SetOperationStmt.analyze()'.
+            Preconditions.checkState(!compatibilityLevel.isUnsafe());
+
+            Preconditions.checkState(compatibleType.isChar());
+            int length = ((ScalarType) compatibleType).getLength();
+            compatibleType = ScalarType.createVarcharType(length);
+            expr = expr.castTo(compatibleType, compatibilityLevel);
+          }
+
+          widestExprs.set(i, expr);
+        }
       }
       // Now that we've found a compatible type, add implicit casts if necessary.
       for (int j = 0; j < exprLists.size(); ++j) {
@@ -4412,4 +4434,11 @@ public class Analyzer {
     }
     return hasNullRejectingTid;
   }
+
+  private static boolean differentLenCharTypes(Type t1, Type t2) {
+    if (!t1.isChar() || !t2.isChar()) return false;
+    ScalarType t1Scalar = (ScalarType) t1;
+    ScalarType t2Scalar = (ScalarType) t2;
+    return t1Scalar.getLength() != t2Scalar.getLength();
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java b/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
index faf0a1324..2353a3494 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
@@ -315,6 +315,15 @@ public class SetOperationStmt extends QueryStmt {
     if (isAnalyzed()) return;
     super.analyze(analyzer);
 
+    final org.apache.impala.thrift.TQueryOptions query_options =
+        analyzer.getQueryCtx().client_request.query_options;
+    if (query_options.values_stmt_avoid_lossy_char_padding
+        && query_options.allow_unsafe_casts) {
+      throw new AnalysisException("Query options ALLOW_UNSAFE_CASTS and " +
+          "VALUES_STMT_AVOID_LOSSY_CHAR_PADDING are not allowed to be set at the same " +
+          "time if the query contains set operation(s).");
+    }
+
     // Propagates DISTINCT from right to left.
     propagateDistinct();
 
@@ -343,7 +352,8 @@ public class SetOperationStmt extends QueryStmt {
     for (SetOperand op : operands_) {
       resultExprLists.add(op.getQueryStmt().getResultExprs());
     }
-    widestExprs_ = analyzer.castToSetOpCompatibleTypes(resultExprLists);
+    widestExprs_ = analyzer.castToSetOpCompatibleTypes(resultExprLists,
+        shouldAvoidLossyCharPadding(analyzer));
     // TODO (IMPALA-11018): Currently only UNION ALL is supported for collection types
     //       due to missing handling in BE.
     if (!hasOnlyUnionAllOps()) {
@@ -353,7 +363,6 @@ public class SetOperationStmt extends QueryStmt {
       }
     }
 
-
     // Create tuple descriptor materialized by this UnionStmt, its resultExprs, and
     // its sortInfo if necessary.
     createMetadata(analyzer);
@@ -381,6 +390,18 @@ public class SetOperationStmt extends QueryStmt {
     baseTblResultExprs_ = resultExprs_;
   }
 
+  /**
+   * If all values in a column are CHARs but they have different lengths, the common type
+   * will normally be the CHAR type of the greatest length, in which case other CHAR
+   * values are padded; this function decides whether this should be avoided by using
+   * VARCHAR as the common type. See IMPALA-10753.
+   *
+   * The default behaviour is returning false, subclasses can override it.
+   */
+  protected boolean shouldAvoidLossyCharPadding(Analyzer analyzer) {
+    return false;
+  }
+
   /**
    * Analyzes all operands and checks that they return an equal number of exprs.
    * Throws an AnalysisException if that is not the case, or if analyzing
diff --git a/fe/src/main/java/org/apache/impala/analysis/StatementBase.java b/fe/src/main/java/org/apache/impala/analysis/StatementBase.java
index 4210f2958..39829b60a 100644
--- a/fe/src/main/java/org/apache/impala/analysis/StatementBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/StatementBase.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Optional;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.TypeCompatibility;
 import org.apache.impala.common.AnalysisException;
@@ -196,14 +197,39 @@ public abstract class StatementBase extends StmtNode {
    * message.
    *
    * 'widestTypeSrcExpr' is the first widest type expression of the source expressions.
-   * This is only used when constructing an AnalysisException message to make sure the
-   * right expression is blamed in the error message.
    *
    * If compatibility is unsafe and the source expression is not constant, compatibility
    * ignores the unsafe option.
    */
   public static Expr checkTypeCompatibility(String dstTableName, Column dstCol,
       Expr srcExpr, Analyzer analyzer, Expr widestTypeSrcExpr) throws AnalysisException {
+    // In 'ValueStmt', if all values in a column are CHARs but they have different
+    // lengths, they are implicitly cast to VARCHAR to avoid padding (see IMPALA-10753 and
+    // ValuesStmt.java). Since VARCHAR cannot normally be cast to CHAR because of possible
+    // loss of precision, if the destination column is CHAR, we have to manually cast the
+    // values back to CHAR. There is no danger of loss of precision here because we cast
+    // the values to the CHAR type of the same length as the VARCHAR type. If that is
+    // still not compatible with the destination column, we throw an AnalysisException.
+    if (widestTypeSrcExpr != null &&  widestTypeSrcExpr.isImplicitCast()) {
+      // TODO: Can we avoid casting CHAR -> VARCHAR -> CHAR for CHAR dst columns?
+      Expr exprWithoutImplicitCast = widestTypeSrcExpr.ignoreImplicitCast();
+      if (srcExpr.getType().isVarchar() && exprWithoutImplicitCast.getType().isChar()) {
+        ScalarType varcharType = (ScalarType) srcExpr.getType();
+        ScalarType charType = (ScalarType) exprWithoutImplicitCast.getType();
+        Preconditions.checkState(varcharType.getLength() >= charType.getLength());
+        Type newCharType = ScalarType.createCharType(varcharType.getLength());
+        Expr newCharExpr = new CastExpr(newCharType, srcExpr);
+        return checkTypeCompatibilityHelper(dstTableName, dstCol, newCharExpr,
+            analyzer, null);
+      }
+    }
+
+    return checkTypeCompatibilityHelper(dstTableName, dstCol, srcExpr, analyzer,
+        widestTypeSrcExpr);
+  }
+
+  private static Expr checkTypeCompatibilityHelper(String dstTableName, Column dstCol,
+      Expr srcExpr, Analyzer analyzer, Expr widestTypeSrcExpr) throws AnalysisException {
     Type dstColType = dstCol.getType();
     Type srcExprType = srcExpr.getType();
 
diff --git a/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
index 349c49188..6231497dc 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
@@ -89,4 +89,10 @@ public class ValuesStmt extends UnionStmt {
    */
   @Override
   public void rewriteExprs(ExprRewriter rewriter) {}
+
+  @Override
+  protected boolean shouldAvoidLossyCharPadding(Analyzer analyzer) {
+    return analyzer.getQueryCtx().client_request
+        .query_options.values_stmt_avoid_lossy_char_padding;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/Type.java b/fe/src/main/java/org/apache/impala/catalog/Type.java
index b0952f0da..b30036ae3 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Type.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Type.java
@@ -188,6 +188,7 @@ public abstract class Type {
   public boolean isDate() { return isScalarType(PrimitiveType.DATE); }
   public boolean isDecimal() { return isScalarType(PrimitiveType.DECIMAL); }
   public boolean isFullySpecifiedDecimal() { return false; }
+  public boolean isChar() { return isScalarType(PrimitiveType.CHAR); }
   public boolean isVarchar() { return isScalarType(PrimitiveType.VARCHAR); }
   public boolean isString() { return isScalarType(PrimitiveType.STRING); }
   public boolean isBinary() { return isScalarType(PrimitiveType.BINARY); }
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
index 4231e649c..ac43508cb 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -362,7 +362,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
 
       // Cast partition exprs of all hash-partitioning senders to their compatible types.
       try {
-        analyzer.castToSetOpCompatibleTypes(senderPartitionExprs);
+        analyzer.castToSetOpCompatibleTypes(senderPartitionExprs, false);
       } catch (AnalysisException e) {
         // Should never happen. Analysis should have ensured type compatibility already.
         throw new IllegalStateException(e);
diff --git a/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-lossy-char-padding.test b/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-lossy-char-padding.test
new file mode 100644
index 000000000..fc78bc32d
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-lossy-char-padding.test
@@ -0,0 +1,33 @@
+====
+---- QUERY
+insert overwrite test_char_values_string_col_tmp values
+(cast("1" as char(1))),
+(cast("12" as char(2))),
+(cast("123" as char(3)));
+---- RESULTS
+: 3
+====
+---- QUERY
+select length(s) from test_char_values_string_col_tmp;
+---- TYPES
+int
+---- RESULTS
+3
+3
+3
+====
+---- QUERY
+insert overwrite test_char_values_mixed_cols_tmp values
+(cast("1" as char(1)), cast("str" as char(3)), cast("vchar" as char(5))),
+(cast("10" as char(4)), cast("str_2" as char(5)), cast("vchar_2" as char(7)))
+---- RESULTS
+: 2
+====
+---- QUERY
+select c, s, v from test_char_values_mixed_cols_tmp;
+---- TYPES
+char,string,string
+---- RESULTS
+'1    ','str  ','vchar  '
+'10   ','str_2','vchar_2'
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-no-lossy-char-padding.test b/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-no-lossy-char-padding.test
new file mode 100644
index 000000000..cd53170dd
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/chars-values-stmt-no-lossy-char-padding.test
@@ -0,0 +1,173 @@
+====
+---- QUERY
+create table test_char_values_string_col_tmp (s string);
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+insert overwrite test_char_values_string_col_tmp values
+(cast("1" as char(1))),
+(cast("12" as char(2))),
+(cast("123" as char(3)));
+---- RESULTS
+: 3
+====
+---- QUERY
+select length(s) from test_char_values_string_col_tmp;
+---- TYPES
+int
+---- RESULTS
+1
+2
+3
+====
+---- QUERY
+create table test_char_values_varchar_col_tmp (v varchar(5));
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+insert overwrite test_char_values_varchar_col_tmp values
+(cast("1" as char(1))),
+(cast("12" as char(2)));
+---- RESULTS
+: 2
+====
+---- QUERY
+select v from test_char_values_varchar_col_tmp;
+---- TYPES
+string
+---- RESULTS
+'1'
+'12'
+====
+---- QUERY
+# A CHAR(10) value can't be inserted into a VARCHAR(5) field.
+insert overwrite test_char_values_varchar_col_tmp values
+(cast("1" as char(1))),
+(cast("0123456789" as char(10)));
+---- CATCH
+AnalysisException: Possible loss of precision for target table
+====
+---- QUERY
+create table test_char_values_char_col_tmp (c char(5));
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+insert overwrite test_char_values_char_col_tmp values
+(cast("1" as char(1))),
+(cast("12" as char(2)));
+---- RESULTS
+: 2
+====
+---- QUERY
+select c from test_char_values_char_col_tmp;
+---- TYPES
+char
+---- RESULTS
+'1    '
+'12   '
+====
+---- QUERY
+create table test_char_values_mixed_cols_tmp (c char(5), s string, v varchar(8));
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+insert overwrite test_char_values_mixed_cols_tmp values
+(cast("1" as char(1)), cast("str" as char(3)), cast("vchar" as char(5))),
+(cast("10" as char(4)), cast("str_2" as char(5)), cast("vchar_2" as char(7)))
+---- RESULTS
+: 2
+====
+---- QUERY
+select c, s, v from test_char_values_mixed_cols_tmp;
+---- TYPES
+char,string,string
+---- RESULTS
+'1    ','str','vchar'
+'10   ','str_2','vchar_2'
+====
+---- QUERY
+create table test_char_values_mixed_cols_different_order_tmp (s string, c char(5), v varchar(8));
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+insert overwrite test_char_values_mixed_cols_different_order_tmp values
+(cast("str" as char(3)), cast("1" as char(1)), cast("vchar" as char(5))),
+(cast("str_2" as char(5)), cast("10" as char(4)), cast("vchar_2" as char(7)))
+---- RESULTS
+: 2
+====
+---- QUERY
+select s, c, v from test_char_values_mixed_cols_different_order_tmp;
+---- TYPES
+string,char,string
+---- RESULTS
+'str','1    ','vchar'
+'str_2','10   ','vchar_2'
+====
+---- QUERY
+# In the first column we have CHARs with the same length, but in the third we don't. The
+# third column should not be padded.
+insert overwrite test_char_values_mixed_cols_different_order_tmp values
+(cast("str" as char(3)), cast("1" as char(1)), cast("vchar" as char(5))),
+(cast("str" as char(3)), cast("10" as char(4)), cast("vchar_2" as char(7)))
+---- RESULTS
+: 2
+====
+---- QUERY
+select s, c, v from test_char_values_mixed_cols_different_order_tmp;
+---- TYPES
+string,char,string
+---- RESULTS
+'str','1    ','vchar'
+'str','10   ','vchar_2'
+====
+---- QUERY
+# In the first column we don't only insert CHARs, but in the third we do. The third column
+# should not be padded.
+insert overwrite test_char_values_mixed_cols_different_order_tmp values
+(cast("str" as char(3)), cast("1" as char(1)), cast("vchar" as char(5))),
+(cast("str_1" as char(5)), cast("1" as char(1)), cast("vchar" as char(5))),
+("str_2", cast("10" as char(4)), cast("vchar_2" as char(7)))
+---- RESULTS
+: 3
+====
+---- QUERY
+select s, c, v from test_char_values_mixed_cols_different_order_tmp;
+---- TYPES
+string,char,string
+---- RESULTS
+'str','1    ','vchar'
+'str_1','1    ','vchar'
+'str_2','10   ','vchar_2'
+====
+---- QUERY
+# In the first column we don't only have CHARs, so the values there should not be cast.
+# In the second column we have only chars so the values should be cast to VARCHAR.
+select typeof(mixed), typeof(chars) from (values
+(cast("str" as char(3)) mixed,   cast("1" as char(1)) chars),
+(cast("str_1" as char(5)),       cast("1" as char(1))),
+("str_2",                        cast("10" as char(4)))
+) vals;
+---- TYPES
+string,string
+---- RESULTS
+'STRING','VARCHAR(4)'
+'STRING','VARCHAR(4)'
+'STRING','VARCHAR(4)'
+====
+
+---- QUERY
+# Setting ALLOW_UNSAFE_CASTS together with VALUES_STMT_AVOID_LOSSY_CHAR_PADDING is not
+# allowed if the query contains set operation(s).
+set ALLOW_UNSAFE_CASTS=1;
+insert overwrite test_char_values_varchar_col_tmp values
+(cast("1" as char(1))),
+(cast("12" as char(2)));
+---- CATCH
+AnalysisException: Query options ALLOW_UNSAFE_CASTS and VALUES_STMT_AVOID_LOSSY_CHAR_PADDING are not allowed to be set at the same time if the query contains set operation(s).
+====
diff --git a/tests/query_test/test_chars.py b/tests/query_test/test_chars.py
index e4cf8ea9f..6f2176600 100644
--- a/tests/query_test/test_chars.py
+++ b/tests/query_test/test_chars.py
@@ -16,6 +16,8 @@
 # under the License.
 
 from __future__ import absolute_import, division, print_function
+from copy import deepcopy
+
 import pytest
 
 from tests.common.impala_test_suite import ImpalaTestSuite
@@ -50,6 +52,19 @@ class TestStringQueries(ImpalaTestSuite):
     # Tests that create temporary tables and require a unique database.
     self.run_test_case('QueryTest/chars-tmp-tables', vector, unique_database)
 
+  # Regression tests for IMPALA-10753.
+  def test_chars_values_stmt(self, vector, unique_database):
+    if vector.get_value('protocol') in ['hs2', 'hs2-http']:
+      pytest.skip("HS2 does not return row counts for inserts")
+    vector = deepcopy(vector)
+    vector.get_value('exec_option')['values_stmt_avoid_lossy_char_padding'] = True
+    self.run_test_case('QueryTest/chars-values-stmt-no-lossy-char-padding',
+        vector, unique_database)
+
+    vector.get_value('exec_option')['values_stmt_avoid_lossy_char_padding'] = False
+    self.run_test_case('QueryTest/chars-values-stmt-lossy-char-padding',
+        vector, unique_database)
+
 class TestCharFormats(ImpalaTestSuite):
   @classmethod
   def get_workload(cls):