You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Shubham Chaurasia (Jira)" <ji...@apache.org> on 2021/06/14 10:18:00 UTC

[jira] [Created] (HIVE-25243) Llap external client - Handle nested null values in struct vector in arrow serializer

Shubham Chaurasia created HIVE-25243:
----------------------------------------

             Summary: Llap external client - Handle nested null values in struct vector in arrow serializer
                 Key: HIVE-25243
                 URL: https://issues.apache.org/jira/browse/HIVE-25243
             Project: Hive
          Issue Type: Bug
          Components: Serializers/Deserializers
            Reporter: Shubham Chaurasia
            Assignee: Shubham Chaurasia


Consider the following table in text format - 
{code}
+-------------------------------+
|              c8               |
+-------------------------------+
| NULL                          |
| {"r":null,"s":null,"t":null}  |
| {"r":"a","s":9,"t":2.2}       |
+-------------------------------+
{code}

When we query above table via llap external client, it throws following exception - 
{code:java}
Caused by: java.lang.NullPointerException: src
        at io.netty.util.internal.ObjectUtil.checkNotNull(ObjectUtil.java:33)
        at io.netty.buffer.UnsafeByteBufUtil.setBytes(UnsafeByteBufUtil.java:537)
        at io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:199)
        at io.netty.buffer.WrappedByteBuf.setBytes(WrappedByteBuf.java:486)
        at io.netty.buffer.UnsafeDirectLittleEndian.setBytes(UnsafeDirectLittleEndian.java:34)
        at io.netty.buffer.ArrowBuf.setBytes(ArrowBuf.java:933)
        at org.apache.arrow.vector.BaseVariableWidthVector.setBytes(BaseVariableWidthVector.java:1191)
        at org.apache.arrow.vector.BaseVariableWidthVector.setSafe(BaseVariableWidthVector.java:1026)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.lambda$static$15(Serializer.java:834)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.writeGeneric(Serializer.java:777)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.writePrimitive(Serializer.java:581)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.write(Serializer.java:290)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.writeStruct(Serializer.java:359)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.write(Serializer.java:296)
        at org.apache.hadoop.hive.ql.io.arrow.Serializer.serializeBatch(Serializer.java:213)
        at org.apache.hadoop.hive.ql.exec.vector.filesink.VectorFileSinkArrowOperator.process(VectorFileSinkArrowOperator.java:135)
{code}

Created a test to repro it - 
{code:java}
/**
 * TestMiniLlapVectorArrowWithLlapIODisabled - turns off llap io while testing LLAP external client flow.
 * The aim of turning off LLAP IO is -
 * when we create table through this test, LLAP caches them and returns the same
 * when we do a read query, due to this we miss some code paths which may have been hit otherwise.
 */
public class TestMiniLlapVectorArrowWithLlapIODisabled extends BaseJdbcWithMiniLlap {

  @BeforeClass
  public static void beforeTest() throws Exception {
    HiveConf conf = defaultConf();
    conf.setBoolVar(ConfVars.LLAP_OUTPUT_FORMAT_ARROW, true);
    conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_FILESINK_ARROW_NATIVE_ENABLED, true);
    conf.set(ConfVars.LLAP_IO_ENABLED.varname, "false");
    BaseJdbcWithMiniLlap.beforeTest(conf);
  }

  @Override
  protected InputFormat<NullWritable, Row> getInputFormat() {
    //For unit testing, no harm in hard-coding allocator ceiling to LONG.MAX_VALUE
    return new LlapArrowRowInputFormat(Long.MAX_VALUE);
  }

  @Test
  public void testNullsInStructFields() throws Exception {
    createDataTypesTable("datatypes");
    RowCollector2 rowCollector = new RowCollector2();
    // c8 struct<r:string,s:int,t:double>
    String query = "select c8 from datatypes";
    int rowCount = processQuery(query, 1, rowCollector);
    assertEquals(3, rowCount);
  }
}
{code}


Cause - As we see in the table above, first row of the table is NULL, and correspondingly we get {{structVector.isNull[i]=true}} in arrow serializer but we don't get {{isNull[i]=true}} for the fields of struct. And later the code goes for setting such fields in arrow vector and we see above exception.






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