qiang.bi created HIVE-25193:
-------------------------------

             Summary: Vectorized Query Execution: ClassCastException when use 
nvl() function which default_value is decimal type
                 Key: HIVE-25193
                 URL: https://issues.apache.org/jira/browse/HIVE-25193
             Project: Hive
          Issue Type: Bug
          Components: Vectorization
    Affects Versions: 4.0.0
            Reporter: qiang.bi
            Assignee: qiang.bi


Problem statement:

 
{code:java}
set hive.vectorized.execution.enabled = true;
select nvl(get_json_object(attr_json,'$.correctedPrice'),0.88) corrected_price,
from dw_mdm_sync_asset;
{code}
 

 

The error log:

 
{code:java}
[2021-05-24 08:06:05.627]], TaskAttempt 3 failed, info=[Error: Error while 
running task ( failure ) : 
attempt_1619882873092_4567_1_03_000000_3:java.lang.RuntimeException: 
java.lang.RuntimeException: Hive Runtime Error while closing 
operators[2021-05-24 08:06:05.627]], TaskAttempt 3 failed, info=[Error: Error 
while running task ( failure ) : 
attempt_1619882873092_4567_1_03_000000_3:java.lang.RuntimeException: 
java.lang.RuntimeException: Hive Runtime Error while closing 
operators[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250)[2021-05-24
 08:06:05.628] at 
org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)[2021-05-24
 08:06:05.628] at 
org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)[2021-05-24
 08:06:05.628] at 
org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)[2021-05-24
 08:06:05.628] at java.security.AccessController.doPrivileged(Native 
Method)[2021-05-24 08:06:05.628] at 
javax.security.auth.Subject.doAs(Subject.java:422)[2021-05-24 08:06:05.628] at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730)[2021-05-24
 08:06:05.628] at 
org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)[2021-05-24
 08:06:05.628] at 
org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)[2021-05-24
 08:06:05.628] at 
org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)[2021-05-24 
08:06:05.628] at 
com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)[2021-05-24
 08:06:05.628] at 
com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)[2021-05-24
 08:06:05.628] at 
com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)[2021-05-24
 08:06:05.628] at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)[2021-05-24
 08:06:05.628] at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)[2021-05-24
 08:06:05.628] at java.lang.Thread.run(Thread.java:748)[2021-05-24 
08:06:05.628]Caused by: java.lang.RuntimeException: Hive Runtime Error while 
closing operators[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.close(MapRecordProcessor.java:495)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)[2021-05-24
 08:06:05.628] ... 16 more[2021-05-24 08:06:05.628]Caused by: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: Error evaluating 
null[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.MapJoinOperator.closeOp(MapJoinOperator.java:621)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinGenerateResultOperator.closeOp(VectorMapJoinGenerateResultOperator.java:681)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)[2021-05-24 
08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:756)[2021-05-24 
08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.close(MapRecordProcessor.java:484)[2021-05-24
 08:06:05.628] ... 17 more[2021-05-24 08:06:05.628]Caused by: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: Error evaluating 
null[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinGenerateResultOperator.reProcessBigTable(VectorMapJoinGenerateResultOperator.java:609)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.MapJoinOperator.continueProcess(MapJoinOperator.java:681)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.MapJoinOperator.closeOp(MapJoinOperator.java:614)[2021-05-24
 08:06:05.628] ... 21 more[2021-05-24 08:06:05.628]Caused by: 
org.apache.hadoop.hive.ql.metadata.HiveException: 
org.apache.hadoop.hive.ql.metadata.HiveException: Error evaluating 
null[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyStringOperator.process(VectorMapJoinInnerBigOnlyStringOperator.java:391)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinGenerateResultOperator.reProcessBigTable(VectorMapJoinGenerateResultOperator.java:602)[2021-05-24
 08:06:05.628] ... 23 more[2021-05-24 08:06:05.628]Caused by: 
org.apache.hadoop.hive.ql.metadata.HiveException: Error evaluating 
null[2021-05-24 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator.process(VectorSelectOperator.java:149)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.Operator.vectorForward(Operator.java:965)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:938)[2021-05-24 
08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinGenerateResultOperator.forwardBigTableBatch(VectorMapJoinGenerateResultOperator.java:640)[2021-05-24
 08:06:05.628] at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyStringOperator.process(VectorMapJoinInnerBigOnlyStringOperator.java:385)[2021-05-24
 08:06:05.629] ... 24 more[2021-05-24 08:06:05.629]Caused by: 
java.lang.ClassCastException: 
org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector cannot be cast to 
org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector[2021-05-24 
08:06:05.629] at 
org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector.setElement(BytesColumnVector.java:504)[2021-05-24
 08:06:05.629] at 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorCoalesce.evaluate(VectorCoalesce.java:124)[2021-05-24
 08:06:05.629] at 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression.evaluateChildren(VectorExpression.java:271)[2021-05-24
 08:06:05.629] at 
org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToDouble.evaluate(CastStringToDouble.java:83)[2021-05-24
 08:06:05.629] at 
org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator.process(VectorSelectOperator.java:146)[2021-05-24
 08:06:05.632] ... 28 more{code}
 

The problem HiveQL:

 
{code:java}
nvl(get_json_object(attr_json,'$.correctedPrice'),0.88) corrected_price
{code}
 

The problem expression:

 
{code:java}
CastStringToDouble(col 39:string)(children: VectorCoalesce(columns [37, 
38])(children: VectorUDFAdaptor(get_json_object(_col14, '$.correctedPrice')) -> 
37:string, ConstantVectorExpression(val 0.88) -> 38:decimal(2,2)) -> 39:string) 
-> 40:double
{code}
 

The problem code:
{code:java}
public class VectorCoalesce extends VectorExpression {  
  ...   
  @Override
  public void evaluate(VectorizedRowBatch batch) throws HiveException {    if 
(childExpressions != null) {
      super.evaluateChildren(batch);
    }    int[] sel = batch.selected;
    int n = batch.size;
    ColumnVector outputColVector = batch.cols[outputColumnNum];
    boolean[] outputIsNull = outputColVector.isNull;
    if (n <= 0) {
      // Nothing to do
      return;
    }    if (unassignedBatchIndices == null || n > 
unassignedBatchIndices.length) {      // (Re)allocate larger to be a multiple 
of 1024 (DEFAULT_SIZE).
      final int roundUpSize =
          ((n + VectorizedRowBatch.DEFAULT_SIZE - 1) / 
VectorizedRowBatch.DEFAULT_SIZE)
              * VectorizedRowBatch.DEFAULT_SIZE;
      unassignedBatchIndices = new int[roundUpSize];
    }    // We do not need to do a column reset since we are carefully changing 
the output.
    outputColVector.isRepeating = false;    // CONSIDER: Should be do this for 
all vector expressions that can
    //           work on BytesColumnVector output columns???
    outputColVector.init();
    final int columnCount = inputColumns.length;    /*
     * Process the input columns to find a non-NULL value for each row.
     *
     * We track the unassigned batchIndex of the rows that have not received
     * a non-NULL value yet.  Similar to a selected array.
     */
    boolean isAllUnassigned = true;
    int unassignedColumnCount = 0;
    for (int k = 0; k < inputColumns.length; k++) {
      ColumnVector cv = batch.cols[inputColumns[k]];
      if (cv.isRepeating) {        if (cv.noNulls || !cv.isNull[0]) {
          /*
           * With a repeating value we can finish all remaining rows.
           */
          if (isAllUnassigned) {            // No other columns provided 
non-NULL values.  We can return repeated output.
            outputIsNull[0] = false;
            outputColVector.setElement(0, 0, cv);
            outputColVector.isRepeating = true;
            return;
          } else {            // Some rows have already been assigned values. 
Assign the remaining.
            // We cannot use copySelected method here.
            for (int i = 0; i < unassignedColumnCount; i++) {
              final int batchIndex = unassignedBatchIndices[i];
              outputIsNull[batchIndex] = false;              // Our input is 
repeating (i.e. inputColNumber = 0).
              outputColVector.setElement(batchIndex, 0, cv);
            }
            return;
          }
        } else {          // Repeated NULLs -- skip this input column.
        }
      } else {        /*
         * Non-repeating input column. Use any non-NULL values for unassigned 
rows.
         */
        if (isAllUnassigned) {          /*
           * No other columns provided non-NULL values.  We *may* be able to 
finish all rows
           * with this input column...
           */
          if (cv.noNulls){            // Since no NULLs, we can provide values 
for all rows.
            if (batch.selectedInUse) {
              for (int i = 0; i < n; i++) {
                final int batchIndex = sel[i];
                outputIsNull[batchIndex] = false;
                outputColVector.setElement(batchIndex, batchIndex, cv);
              }
            } else {
              Arrays.fill(outputIsNull, 0, n, false);
              for (int batchIndex = 0; batchIndex < n; batchIndex++) {
                outputColVector.setElement(batchIndex, batchIndex, cv);
              }
            }
            return;
          } else {            // We might not be able to assign all rows 
because of input NULLs.  Start tracking any
            // unassigned rows.
            boolean[] inputIsNull = cv.isNull;
            if (batch.selectedInUse) {
              for (int i = 0; i < n; i++) {
                final int batchIndex = sel[i];
                if (!inputIsNull[batchIndex]) {
                  outputIsNull[batchIndex] = false;
                  outputColVector.setElement(batchIndex, batchIndex, cv);
                } else {
                  unassignedBatchIndices[unassignedColumnCount++] = batchIndex;
                }
              }
            } else {
              for (int batchIndex = 0; batchIndex < n; batchIndex++) {
                if (!inputIsNull[batchIndex]) {
                  outputIsNull[batchIndex] = false;
                  outputColVector.setElement(batchIndex, batchIndex, cv);
                } else {
                  unassignedBatchIndices[unassignedColumnCount++] = batchIndex;
                }
              }
            }
            if (unassignedColumnCount == 0) {
              return;
            }
            isAllUnassigned = false;
          }
        } else {          /*
           * We previously assigned *some* rows with non-NULL values. The batch 
indices of
           * the unassigned row were tracked.
           */
          if (cv.noNulls) {            // Assign all remaining rows.
            for (int i = 0; i < unassignedColumnCount; i++) {
              final int batchIndex = unassignedBatchIndices[i];
              outputIsNull[batchIndex] = false;
              outputColVector.setElement(batchIndex, batchIndex, cv);
            }
            return;
          } else {            // Use any non-NULL values found; remember the 
remaining unassigned.
            boolean[] inputIsNull = cv.isNull;
            int newUnassignedColumnCount = 0;
            for (int i = 0; i < unassignedColumnCount; i++) {
              final int batchIndex = unassignedBatchIndices[i];
              if (!inputIsNull[batchIndex]) {
                outputIsNull[batchIndex] = false;
                outputColVector.setElement(batchIndex, batchIndex, cv);
              } else {
                unassignedBatchIndices[newUnassignedColumnCount++] = batchIndex;
              }
            }
            if (newUnassignedColumnCount == 0) {
              return;
            }
            unassignedColumnCount = newUnassignedColumnCount;
          }
        }
      }
    }    // NULL out the remaining columns.
    outputColVector.noNulls = false;
    if (isAllUnassigned) {
      outputIsNull[0] = true;
      outputColVector.isRepeating = true;
    } else {
      for (int i = 0; i < unassignedColumnCount; i++) {
        final int batchIndex = unassignedBatchIndices[i];
        outputIsNull[batchIndex] = true;
      }
    }
  }
  ...
}
{code}
The above code,  outputColVector is BytesColumnVector type, but one of 

the columnVector is DecimalColumnVector type.

 

At present, we can add single quotes in “0.88” to resolve this problem.For 
example: 
{code:java}
nvl(get_json_object(attr_json,'$.correctedPrice'), '0.88') corrected_price
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to