You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2014/10/09 03:10:13 UTC

svn commit: r1630265 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java

Author: prasadm
Date: Thu Oct  9 01:10:12 2014
New Revision: 1630265

URL: http://svn.apache.org/r1630265
Log:
HIVE-7932: It may cause NP exception when add accessed columns to ReadEntity (Xiaomeng Huang via Prasad Mujumdar)

Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1630265&r1=1630264&r2=1630265&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Oct  9 01:10:12 2014
@@ -10048,14 +10048,19 @@ public class SemanticAnalyzer extends Ba
     Map<String, List<String>> tableToColumnAccessMap = columnAccessInfo.getTableToColumnAccessMap();
     if (tableToColumnAccessMap != null && !tableToColumnAccessMap.isEmpty()) {
       for(ReadEntity entity: inputs) {
+        List<String> cols;
         switch (entity.getType()) {
           case TABLE:
-            entity.getAccessedColumns().addAll(
-                tableToColumnAccessMap.get(entity.getTable().getCompleteName()));
+            cols = tableToColumnAccessMap.get(entity.getTable().getCompleteName());
+            if (cols != null && !cols.isEmpty()) {
+              entity.getAccessedColumns().addAll(cols);
+            }
             break;
           case PARTITION:
-            entity.getAccessedColumns().addAll(
-                tableToColumnAccessMap.get(entity.getPartition().getTable().getCompleteName()));
+            cols = tableToColumnAccessMap.get(entity.getPartition().getTable().getCompleteName());
+            if (cols != null && !cols.isEmpty()) {
+              entity.getAccessedColumns().addAll(cols);
+            }
             break;
           default:
             // no-op

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java?rev=1630265&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java Thu Oct  9 01:10:12 2014
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.parse;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestColumnAccess {
+
+  @BeforeClass
+  public static void Setup() throws CommandNeedRetryException {
+    Driver driver = createDriver();
+    int ret = driver.run("create table t1(id1 int, name1 string)").getResponseCode();
+    Assert.assertEquals("Checking command success", 0, ret);
+    ret = driver.run("create table t2(id2 int, id1 int, name2 string)").getResponseCode();
+    Assert.assertEquals("Checking command success", 0, ret);
+    ret = driver.run("create view v1 as select * from t1").getResponseCode();
+    Assert.assertEquals("Checking command success", 0, ret);
+  }
+
+  @Test
+  public void testQueryTable1() throws ParseException {
+    String query = "select * from t1";
+    Driver driver = createDriver();
+    int rc = driver.compile(query);
+    Assert.assertEquals("Checking command success", 0, rc);
+    QueryPlan plan = driver.getPlan();
+    // check access columns from ColumnAccessInfo
+    ColumnAccessInfo columnAccessInfo = plan.getColumnAccessInfo();
+    List<String> cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+
+    // check access columns from readEntity
+    Map<String, List<String>> tableColsMap = getColsFromReadEntity(plan.getInputs());
+    cols = tableColsMap.get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+  }
+
+  @Test
+  public void testJoinTable1AndTable2() throws ParseException {
+    String query = "select * from t1 join t2 on (t1.id1 = t2.id1)";
+    Driver driver = createDriver();
+    int rc = driver.compile(query);
+    Assert.assertEquals("Checking command success", 0, rc);
+    QueryPlan plan = driver.getPlan();
+    // check access columns from ColumnAccessInfo
+    ColumnAccessInfo columnAccessInfo = plan.getColumnAccessInfo();
+    List<String> cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+    cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t2");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(3, cols.size());
+    Assert.assertNotNull(cols.contains("id2"));
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+
+
+    // check access columns from readEntity
+    Map<String, List<String>> tableColsMap = getColsFromReadEntity(plan.getInputs());
+    cols = tableColsMap.get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+    cols = tableColsMap.get("default@t2");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(3, cols.size());
+    Assert.assertNotNull(cols.contains("id2"));
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+  }
+
+  @Test
+  public void testJoinView1AndTable2() throws ParseException {
+    String query = "select * from v1 join t2 on (v1.id1 = t2.id1)";
+    Driver driver = createDriver();
+    int rc = driver.compile(query);
+    Assert.assertEquals("Checking command success", 0, rc);
+    QueryPlan plan = driver.getPlan();
+    // check access columns from ColumnAccessInfo
+    ColumnAccessInfo columnAccessInfo = plan.getColumnAccessInfo();
+    List<String> cols = columnAccessInfo.getTableToColumnAccessMap().get("default@v1");
+    Assert.assertNull(cols);
+    cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+    cols = columnAccessInfo.getTableToColumnAccessMap().get("default@t2");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(3, cols.size());
+    Assert.assertNotNull(cols.contains("id2"));
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+
+
+    // check access columns from readEntity
+    Map<String, List<String>> tableColsMap = getColsFromReadEntity(plan.getInputs());
+    cols = tableColsMap.get("default@v1");
+    Assert.assertNull(cols);
+    cols = tableColsMap.get("default@t1");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(2, cols.size());
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+    cols = tableColsMap.get("default@t2");
+    Assert.assertNotNull(cols);
+    Assert.assertEquals(3, cols.size());
+    Assert.assertNotNull(cols.contains("id2"));
+    Assert.assertNotNull(cols.contains("id1"));
+    Assert.assertNotNull(cols.contains("name1"));
+  }
+
+  private Map<String, List<String>> getColsFromReadEntity(HashSet<ReadEntity> inputs) {
+    Map<String, List<String>> tableColsMap = new HashMap<String, List<String>>();
+    for(ReadEntity entity: inputs) {
+      switch (entity.getType()) {
+        case TABLE:
+          if (entity.getAccessedColumns() != null && !entity.getAccessedColumns().isEmpty()) {
+            tableColsMap.put(entity.getTable().getCompleteName(), entity.getAccessedColumns());
+          }
+          break;
+        case PARTITION:
+          if (entity.getAccessedColumns() != null && !entity.getAccessedColumns().isEmpty()) {
+            tableColsMap.put(entity.getPartition().getTable().getCompleteName(),
+                entity.getAccessedColumns());
+          }
+          break;
+        default:
+          // no-op
+      }
+    }
+    return tableColsMap;
+  }
+
+  private static Driver createDriver() {
+    HiveConf conf = new HiveConf(Driver.class);
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS, true);
+    SessionState.start(conf);
+    Driver driver = new Driver(conf);
+    driver.init();
+    return driver;
+  }
+
+}