You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mu...@apache.org on 2021/01/06 12:12:42 UTC

[hive] branch master updated: HIVE-24575: VectorGroupByOperator reusing keys can lead to wrong results (Zhihua Deng via Mustafa Iman)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 832f2e4  HIVE-24575: VectorGroupByOperator reusing keys can lead to wrong results (Zhihua Deng via Mustafa Iman)
832f2e4 is described below

commit 832f2e406e0f3bb23897c239cca972618b6e3f6e
Author: dengzhihua <de...@xiaomi.com>
AuthorDate: Tue Jan 5 18:38:58 2021 +0800

    HIVE-24575: VectorGroupByOperator reusing keys can lead to wrong results (Zhihua Deng via Mustafa Iman)
---
 .../wrapper/VectorHashKeyWrapperGeneral.java       |  3 +-
 .../exec/vector/TestVectorHashKeyWrapperBatch.java | 43 ++++++++++++++++++++++
 2 files changed, 44 insertions(+), 2 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/wrapper/VectorHashKeyWrapperGeneral.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/wrapper/VectorHashKeyWrapperGeneral.java
index 929bb0a..3686cb7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/wrapper/VectorHashKeyWrapperGeneral.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/wrapper/VectorHashKeyWrapperGeneral.java
@@ -348,8 +348,7 @@ public class VectorHashKeyWrapperGeneral extends VectorHashKeyWrapperBase {
         }
       } else {
         System.arraycopy(byteLengths, 0, clone.byteLengths, 0, byteValues.length);
-        Arrays.fill(byteStarts, 0);
-        System.arraycopy(byteStarts, 0, clone.byteStarts, 0, byteValues.length);
+        Arrays.fill(clone.byteStarts, 0);
         for (int i = 0; i < byteValues.length; ++i) {
           // avoid allocation/copy of nulls, because it potentially expensive.
           // branch instead.
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
index d50f6e6..aed0847 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
@@ -27,10 +27,12 @@ import java.sql.Timestamp;
 
 import org.junit.Test;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromObjectIterables;
 import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBase;
 import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperBatch;
+import org.apache.hadoop.hive.ql.exec.vector.wrapper.VectorHashKeyWrapperGeneral;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -85,4 +87,45 @@ public class TestVectorHashKeyWrapperBatch {
     assertFalse(vhk.isNull(0));
     assertEquals(vhk.getTimestamp(0), ts1);
   }
+
+  // Test for HIVE-24575
+  @Test
+  public void testVectorHashKeyWrapperGeneralCopyKey() throws HiveException {
+    VectorExpression[] keyExpressions = new VectorExpression[] { new IdentityExpression(0) };
+    TypeInfo[] typeInfos = new TypeInfo[] {TypeInfoFactory.stringTypeInfo};
+    VectorHashKeyWrapperBatch vhkwb = VectorHashKeyWrapperBatch.compileKeyWrapperBatch(
+        keyExpressions,
+        typeInfos);
+
+    VectorizedRowBatch batch = new VectorizedRowBatch(1);
+    batch.selectedInUse = false;
+    BytesColumnVector bytesColumnVector = new BytesColumnVector();
+    bytesColumnVector.initBuffer(1024);
+    batch.cols[0] = bytesColumnVector;
+
+    byte[] contents = "education_reference".getBytes();
+    bytesColumnVector.setVal(0, "system_management".getBytes());
+    bytesColumnVector.setVal(1, "travel_transportation".getBytes());
+    bytesColumnVector.setVal(2, contents);
+    bytesColumnVector.setVal(3, "app_management".getBytes());
+    batch.size = 4;
+
+    vhkwb.evaluateBatch(batch);
+    VectorHashKeyWrapperBase[] vhkwArray = vhkwb.getVectorHashKeyWrappers();
+    VectorHashKeyWrapperGeneral hashKey2 = (VectorHashKeyWrapperGeneral)vhkwArray[2];
+    VectorHashKeyWrapperGeneral hashKey1 = (VectorHashKeyWrapperGeneral)vhkwArray[1];
+
+    assertTrue(StringExpr.equal(hashKey2.getBytes(0), hashKey2.getByteStart(0), hashKey2.getByteLength(0),
+            contents, 0, contents.length));
+    assertFalse(StringExpr.equal(hashKey2.getBytes(0), hashKey2.getByteStart(0), hashKey2.getByteLength(0),
+        hashKey1.getBytes(0), hashKey1.getByteStart(0), hashKey1.getByteLength(0)));
+
+    hashKey2.copyKey(hashKey1);
+
+    assertTrue(StringExpr.equal(hashKey2.getBytes(0), hashKey2.getByteStart(0), hashKey2.getByteLength(0),
+            contents, 0, contents.length));
+    assertTrue(StringExpr.equal(hashKey2.getBytes(0), hashKey2.getByteStart(0), hashKey2.getByteLength(0),
+        hashKey1.getBytes(0), hashKey1.getByteStart(0), hashKey1.getByteLength(0)));
+  }
+
 }
\ No newline at end of file