You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2018/08/16 22:06:22 UTC

hive git commit: HIVE-19254 : NumberFormatException in MetaStoreUtils.isFastStatsSame (Bharathkrishna Guruvayoor Murali, reviewed by Vihang Karajgaonkar)

Repository: hive
Updated Branches:
  refs/heads/master a72693cc3 -> 8c6b5c66d


HIVE-19254 : NumberFormatException in MetaStoreUtils.isFastStatsSame (Bharathkrishna Guruvayoor Murali, reviewed by Vihang Karajgaonkar)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8c6b5c66
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8c6b5c66
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8c6b5c66

Branch: refs/heads/master
Commit: 8c6b5c66d8442517710c779cf7b3e9adbf7ce019
Parents: a72693c
Author: Bharathkrishna Guruvayoor Murali <bh...@cloudera.com>
Authored: Thu Aug 16 15:06:31 2018 -0700
Committer: Vihang Karajgaonkar <vi...@apache.org>
Committed: Thu Aug 16 15:06:31 2018 -0700

----------------------------------------------------------------------
 .../metastore/utils/MetaStoreServerUtils.java   |  4 +-
 .../metastore/utils/TestMetaStoreUtils.java     | 65 ++++++++++++++++++++
 2 files changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8c6b5c66/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
index 69846b7..23ac7da 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
@@ -369,9 +369,9 @@ public class MetaStoreServerUtils {
 
   public static boolean isFastStatsSame(Partition oldPart, Partition newPart) {
     // requires to calculate stats if new and old have different fast stats
-    if ((oldPart != null) && (oldPart.getParameters() != null)) {
+    if ((oldPart != null) && oldPart.isSetParameters() && newPart != null && newPart.isSetParameters()) {
       for (String stat : StatsSetupConst.FAST_STATS) {
-        if (oldPart.getParameters().containsKey(stat)) {
+        if (oldPart.getParameters().containsKey(stat) && newPart.getParameters().containsKey(stat)) {
           Long oldStat = Long.parseLong(oldPart.getParameters().get(stat));
           Long newStat = Long.parseLong(newPart.getParameters().get(stat));
           if (!oldStat.equals(newStat)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/8c6b5c66/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
index d09ac8c..ffe9449 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreUtils.java
@@ -29,13 +29,17 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.thrift.TException;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -43,14 +47,18 @@ import java.util.List;
 import java.util.Map;
 
 import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE;
+import static org.apache.hadoop.hive.common.StatsSetupConst.FAST_STATS;
 import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_FILES;
 import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_ERASURE_CODED_FILES;
 import static org.apache.hadoop.hive.common.StatsSetupConst.STATS_GENERATED;
 import static org.apache.hadoop.hive.common.StatsSetupConst.TOTAL_SIZE;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.updateTableStatsSlow;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.isFastStatsSame;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
@@ -277,6 +285,63 @@ public class TestMetaStoreUtils {
     verify(wh, never()).getFileStatusesForUnpartitionedTable(db, tbl2);
   }
 
+  @Test
+  public void testIsFastStatsSameHandleNull() throws TException {
+
+    FieldSchema fs = new FieldSchema("date", "string", "date column");
+    List<FieldSchema> cols = Collections.singletonList(fs);
+
+    Table tbl = new TableBuilder()
+        .setDbName(DB_NAME)
+        .setTableName(TABLE_NAME)
+        .addCol("id", "int")
+        .build(null);
+    List<String> vals = new ArrayList<String>();
+    vals.add("val1");
+    Partition oldPart;
+    Partition newPart;
+
+    PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(tbl);
+    vals.forEach(val -> partitionBuilder.addValue(val));
+
+    oldPart = partitionBuilder.build(null);
+    newPart = partitionBuilder.build(null);
+
+    Map<String, String> oldParams = new HashMap<>();
+    Map<String, String> newParams = new HashMap<>();
+
+    //Test case where all parameters are present and their values are same
+    long testVal = 1;
+    for (String key : FAST_STATS) {
+      oldParams.put(key, String.valueOf(testVal));
+      newParams.put(key, String.valueOf(testVal));
+    }
+    oldPart.setParameters(oldParams);
+    newPart.setParameters(newParams);
+    assertTrue(isFastStatsSame(oldPart, newPart));
+
+    //Test case where all parameters are present and their values are different
+    for (String key : FAST_STATS) {
+      oldParams.put(key, String.valueOf(testVal));
+      newParams.put(key, String.valueOf(++testVal));
+    }
+    oldPart.setParameters(oldParams);
+    newPart.setParameters(newParams);
+    assertFalse(isFastStatsSame(oldPart, newPart));
+
+    //Test case where newPart is null
+    assertFalse(isFastStatsSame(oldPart, null));
+
+    //Test case where oldPart is null
+    assertFalse(isFastStatsSame(null, newPart));
+
+    //Test case where one or all of the FAST_STATS parameters are not present in newPart
+    Map<String, String> randomParams = new HashMap<String, String>();
+    randomParams.put("randomParam1", "randomVal1");
+    newPart.setParameters(randomParams);
+    assertFalse(isFastStatsSame(oldPart, newPart));
+  }
+
   /**
    * Build a FileStatus object.
    */