XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.13.0
    • None
    • None

    Description

      Query ran:

      select cast(t as timestamp), cast(si as timestamp),
             cast(i as timestamp), cast(b as timestamp),
             cast(f as string), cast(d as timestamp),
             cast(bo as timestamp), cast(b * 0 as timestamp),
             cast(ts as timestamp), cast(s as timestamp),
             cast(substr(s, 1, 1) as timestamp)
      from Table1;
      

      Running this query with hive.vectorized.execution.enabled=true fails with the following exception:

      13/12/05 07:56:36 ERROR tez.TezJobMonitor: Status: Failed
      Vertex failed, vertexName=Map 1, vertexId=vertex_1386227234886_0482_1_00, diagnostics=[Task failed, taskId=task_1386227234886_0482_1_00_000000, diagnostics=[AttemptID:attempt_1386227234886_0482_1_00_000000_0 Info:Error: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row
              at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.processRow(MapRecordProcessor.java:205)
              at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:171)
              at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:112)
              at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:201)
              at org.apache.hadoop.mapred.YarnTezDagChild$4.run(YarnTezDagChild.java:484)
              at java.security.AccessController.doPrivileged(Native Method)
              at javax.security.auth.Subject.doAs(Subject.java:396)
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
              at org.apache.hadoop.mapred.YarnTezDagChild.main(YarnTezDagChild.java:474)
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row
              at org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator.process(VectorMapOperator.java:45)
              at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.processRow(MapRecordProcessor.java:193)
              ... 8 more
      Caused by: java.lang.IllegalArgumentException: nanos > 999999999 or < 0
              at java.sql.Timestamp.setNanos(Timestamp.java:383)
              at org.apache.hadoop.hive.ql.exec.vector.TimestampUtils.assignTimeInNanoSec(TimestampUtils.java:27)
              at org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory$1.writeValue(VectorExpressionWriterFactory.java:412)
              at org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory$VectorExpressionWriterLong.writeValue(VectorExpressionWriterFactory.java:162)
              at org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch.toString(VectorizedRowBatch.java:152)
              at org.apache.hadoop.hive.ql.exec.vector.VectorFileSinkOperator.processOp(VectorFileSinkOperator.java:85)
              at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:786)
              at org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator.processOp(VectorSelectOperator.java:129)
              at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:786)
              at org.apache.hadoop.hive.ql.exec.TableScanOperator.processOp(TableScanOperator.java:93)
              at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:786)
              at org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator.process(VectorMapOperator.java:43)
              ... 9 more
      

      Full log is attached.
      Schema for the table is as follows:

      hive> desc Table1;
      OK
      t                   	tinyint             	from deserializer
      si                  	smallint            	from deserializer
      i                   	int                 	from deserializer
      b                   	bigint              	from deserializer
      f                   	float               	from deserializer
      d                   	double              	from deserializer
      bo                  	boolean             	from deserializer
      s                   	string              	from deserializer
      s2                  	string              	from deserializer
      ts                  	timestamp           	from deserializer
      Time taken: 0.521 seconds, Fetched: 10 row(s)
      

      Attachments

        1. HIVE-5979.1.patch
          4 kB
          Jitendra Nath Pandey
        2. HIVE-5979.2.patch
          5 kB
          Jitendra Nath Pandey

        Activity

          People

            jnp Jitendra Nath Pandey
            jnp Jitendra Nath Pandey
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: