Gopal V created HIVE-12463:
------------------------------

             Summary: VectorMapJoinFastKeyStore has Array OOB errors
                 Key: HIVE-12463
                 URL: https://issues.apache.org/jira/browse/HIVE-12463
             Project: Hive
          Issue Type: Bug
          Components: Vectorization
    Affects Versions: 2.0.0
            Reporter: Gopal V
            Assignee: Gopal V


When combining different sized keys, observing an occasional error in hashtable 
probes.

{code}
Caused by: java.lang.ArrayIndexOutOfBoundsException: 162046429
        at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastKeyStore.equalKey(VectorMapJoinFastKeyStore.java:150)
        at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastBytesHashTable.findReadSlot(VectorMapJoinFastBytesHashTable.java:191)
        at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast.VectorMapJoinFastBytesHashMap.lookup(VectorMapJoinFastBytesHashMap.java:76)
        at 
org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerMultiKeyOperator.process(VectorMapJoinInnerMultiKeyOperator.java:300)
        ... 26 more
{code}

{code}
    // Our reading is positioned to the key.
    writeBuffers.getByteSegmentRefToCurrent(byteSegmentRef, keyLength, readPos);

    byte[] currentBytes = byteSegmentRef.getBytes();
    int currentStart = (int) byteSegmentRef.getOffset();

    for (int i = 0; i < keyLength; i++) {
      if (currentBytes[currentStart + i] != keyBytes[keyStart + i]) {
        // LOG.debug("VectorMapJoinFastKeyStore equalKey no match on bytes");
        return false;
      }
    }
{code}

This needs an identical fix to match 

{code}
    // Rare case of buffer boundary. Unfortunately we'd have to copy some bytes.

   // Rare case of buffer boundary. Unfortunately we'd have to copy some bytes.
    byte[] bytes = new byte[length];
    int destOffset = 0;
    while (destOffset < length) {
      ponderNextBufferToRead(readPos);
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to