You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/08/06 02:10:36 UTC

svn commit: r1510823 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ jdbc/src/test/org/apache/hive/jdbc/ service/src/java/org/apache/hive/service/cli/operation/

Author: gunther
Date: Tue Aug  6 00:10:35 2013
New Revision: 1510823

URL: http://svn.apache.org/r1510823
Log:
HIVE-4573: Support alternate table types for HiveServer2 (Prasad Mujumdar via Gunther Hagleitner)

Added:
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMapping.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1510823&r1=1510822&r2=1510823&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Aug  6 00:10:35 2013
@@ -738,6 +738,7 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_PLAIN_LDAP_DOMAIN("hive.server2.authentication.ldap.Domain", null),
     HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS("hive.server2.custom.authentication.class", null),
     HIVE_SERVER2_ENABLE_DOAS("hive.server2.enable.doAs", true),
+    HIVE_SERVER2_TABLE_TYPE_MAPPING("hive.server2.table.type.mapping", "HIVE"),
 
     HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list", null),
 

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1510823&r1=1510822&r2=1510823&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Tue Aug  6 00:10:35 2013
@@ -1919,6 +1919,17 @@
   </description>
 </property>
 
+<property>
+  <name>hive.server2.table.type.mapping</name>
+  <value>HIVE</value>
+  <description>
+   This setting reflects how HiveServer will report the table types for JDBC and other
+   client implementations that retrieves the available tables and supported table types
+     HIVE : Exposes the hive's native table tyes like MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW
+     CLASSIC : More generic types like TABLE and VIEW
+  </description>
+</property>
+
 
 </configuration>
 

Modified: hive/trunk/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1510823&r1=1510822&r2=1510823&view=diff
==============================================================================
--- hive/trunk/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/trunk/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java Tue Aug  6 00:10:35 2013
@@ -42,7 +42,12 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hive.common.util.HiveVersionInfo;
+import org.apache.hive.service.cli.operation.ClassicTableTypeMapping;
+import org.apache.hive.service.cli.operation.ClassicTableTypeMapping.ClassicTableTypes;
+import org.apache.hive.service.cli.operation.HiveTableTypeMapping;
+import org.apache.hive.service.cli.operation.TableTypeMappingFactory.TableTypeMappings;
 
 
 /**
@@ -664,6 +669,25 @@ public class TestJdbcDriver2 extends Tes
   }
 
   public void testMetaDataGetTables() throws SQLException {
+    getTablesTest(TableType.MANAGED_TABLE.toString(), TableType.VIRTUAL_VIEW.toString());
+  }
+
+  public  void testMetaDataGetTablesHive() throws SQLException {
+    Statement stmt = con.createStatement();
+    stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
+        " = " + TableTypeMappings.HIVE.toString());
+    getTablesTest(TableType.MANAGED_TABLE.toString(), TableType.VIRTUAL_VIEW.toString());
+  }
+
+  public  void testMetaDataGetTablesClassic() throws SQLException {
+    Statement stmt = con.createStatement();
+    stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
+        " = " + TableTypeMappings.CLASSIC.toString());
+    stmt.close();
+    getTablesTest(ClassicTableTypes.TABLE.toString(), ClassicTableTypes.VIEW.toString());
+  }
+
+  private void getTablesTest(String tableTypeName, String viewTypeName) throws SQLException {
     Map<String, Object[]> tests = new HashMap<String, Object[]>();
     tests.put("test%jdbc%", new Object[]{"testhivejdbcdriver_table"
             , "testhivejdbcdriverpartitionedtable"
@@ -698,7 +722,9 @@ public class TestJdbcDriver2 extends Tes
         assertTrue("Missing comment on the table.", resultTableComment.length()>0);
         String tableType = rs.getString("TABLE_TYPE");
         if (resultTableName.endsWith("view")) {
-          assertEquals("Expected a tabletype view but got something else.", "VIRTUAL_VIEW", tableType);
+          assertEquals("Expected a tabletype view but got something else.", viewTypeName, tableType);
+        } else {
+          assertEquals("Expected a tabletype table but got something else.", tableTypeName, tableType);
         }
         cnt++;
       }
@@ -708,7 +734,7 @@ public class TestJdbcDriver2 extends Tes
 
     // only ask for the views.
     ResultSet rs = (ResultSet)con.getMetaData().getTables("default", null, null
-            , new String[]{"VIRTUAL_VIEW"});
+            , new String[]{viewTypeName});
     int cnt=0;
     while (rs.next()) {
       cnt++;
@@ -742,13 +768,28 @@ public class TestJdbcDriver2 extends Tes
   }
 
   public void testMetaDataGetTableTypes() throws SQLException {
-    ResultSet rs = (ResultSet)con.getMetaData().getTableTypes();
+    metaDataGetTableTypeTest(new HiveTableTypeMapping().getTableTypeNames());
+  }
+
+  public void testMetaDataGetHiveTableTypes() throws SQLException {
+    Statement stmt = con.createStatement();
+    stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
+        " = " + TableTypeMappings.HIVE.toString());
+    stmt.close();
+    metaDataGetTableTypeTest(new HiveTableTypeMapping().getTableTypeNames());
+  }
 
-    Set<String> tabletypes = new HashSet();
-    tabletypes.add("MANAGED_TABLE");
-    tabletypes.add("EXTERNAL_TABLE");
-    tabletypes.add("VIRTUAL_VIEW");
-    tabletypes.add("INDEX_TABLE");
+  public void testMetaDataGetClassicTableTypes() throws SQLException {
+    Statement stmt = con.createStatement();
+    stmt.execute("set " + HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING.varname +
+        " = " + TableTypeMappings.CLASSIC.toString());
+    stmt.close();
+    metaDataGetTableTypeTest(new ClassicTableTypeMapping().getTableTypeNames());
+  }
+
+  private void metaDataGetTableTypeTest(Set<String> tabletypes)
+      throws SQLException {
+    ResultSet rs = (ResultSet)con.getMetaData().getTableTypes();
 
     int cnt = 0;
     while (rs.next()) {

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java?rev=1510823&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java Tue Aug  6 00:10:35 2013
@@ -0,0 +1,86 @@
+/**
+ * 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.hive.service.cli.operation;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hive.metastore.TableType;
+
+/**
+ * ClassicTableTypeMapping.
+ * Classic table type mapping :
+ *  Managed Table ==> Table
+ *  External Table ==> Table
+ *  Virtual View ==> View
+ */
+public class ClassicTableTypeMapping implements TableTypeMapping {
+
+  public enum ClassicTableTypes {
+    TABLE,
+    VIEW,
+  }
+
+  private final Map<String, String> hiveToClientMap = new HashMap<String, String>();
+  private final Map<String, String> clientToHiveMap = new HashMap<String, String>();
+
+  public ClassicTableTypeMapping () {
+    hiveToClientMap.put(TableType.MANAGED_TABLE.toString(),
+        ClassicTableTypes.TABLE.toString());
+    hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(),
+        ClassicTableTypes.TABLE.toString());
+    hiveToClientMap.put(TableType.VIRTUAL_VIEW.toString(),
+        ClassicTableTypes.VIEW.toString());
+
+    clientToHiveMap.put(ClassicTableTypes.TABLE.toString(),
+        TableType.MANAGED_TABLE.toString());
+    clientToHiveMap.put(ClassicTableTypes.VIEW.toString(),
+        TableType.VIRTUAL_VIEW.toString());
+  }
+
+  @Override
+  public String mapToHiveType(String clientTypeName) {
+    if (clientToHiveMap.containsKey(clientTypeName)) {
+      return clientToHiveMap.get(clientTypeName);
+    } else {
+      return clientTypeName;
+    }
+  }
+
+  @Override
+  public String mapToClientType(String hiveTypeName) {
+    if (hiveToClientMap.containsKey(hiveTypeName)) {
+      return hiveToClientMap.get(hiveTypeName);
+    } else {
+      return hiveTypeName;
+    }
+  }
+
+  @Override
+  public Set<String> getTableTypeNames() {
+    Set<String> typeNameSet = new HashSet<String>();
+    for (ClassicTableTypes typeNames : ClassicTableTypes.values()) {
+      typeNameSet.add(typeNames.toString());
+    }
+    return typeNameSet;
+  }
+
+}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java?rev=1510823&r1=1510822&r2=1510823&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java Tue Aug  6 00:10:35 2013
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli.operation;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -37,9 +38,14 @@ public class GetTableTypesOperation exte
   .addStringColumn("TABLE_TYPE", "Table type name.");
 
   private RowSet rowSet;
+  private final TableTypeMapping tableTypeMapping;
 
   protected GetTableTypesOperation(HiveSession parentSession) {
     super(parentSession, OperationType.GET_TABLE_TYPES);
+    String tableMappingStr = getParentSession().getHiveConf().
+        getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING);
+    tableTypeMapping =
+      TableTypeMappingFactory.getTableTypeMapping(tableMappingStr);
   }
 
   /* (non-Javadoc)
@@ -51,7 +57,8 @@ public class GetTableTypesOperation exte
     try {
       rowSet = new RowSet();
       for (TableType type : TableType.values()) {
-        rowSet.addRow(RESULT_SET_SCHEMA, new String[] {type.toString()});
+        rowSet.addRow(RESULT_SET_SCHEMA,
+            new String[] {tableTypeMapping.mapToClientType(type.toString())});
       }
       setState(OperationState.FINISHED);
     } catch (Exception e) {

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java?rev=1510823&r1=1510822&r2=1510823&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java Tue Aug  6 00:10:35 2013
@@ -21,6 +21,7 @@ package org.apache.hive.service.cli.oper
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hive.service.cli.FetchOrientation;
@@ -42,6 +43,7 @@ public class GetTablesOperation extends 
   private final String tableName;
   private final List<String> tableTypes = new ArrayList<String>();
   private final RowSet rowSet = new RowSet();
+  private final TableTypeMapping tableTypeMapping;
 
 
   private static final TableSchema RESULT_SET_SCHEMA = new TableSchema()
@@ -58,6 +60,10 @@ public class GetTablesOperation extends 
     this.catalogName = catalogName;
     this.schemaName = schemaName;
     this.tableName = tableName;
+    String tableMappingStr = getParentSession().getHiveConf().
+        getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING);
+    tableTypeMapping =
+        TableTypeMappingFactory.getTableTypeMapping(tableMappingStr);
     if (tableTypes != null) {
       this.tableTypes.addAll(tableTypes);
     }
@@ -80,10 +86,11 @@ public class GetTablesOperation extends 
               DEFAULT_HIVE_CATALOG,
               table.getDbName(),
               table.getTableName(),
-              table.getTableType(),
+              tableTypeMapping.mapToClientType(table.getTableType()),
               table.getParameters().get("comment")
               };
-          if (tableTypes.isEmpty() || tableTypes.contains(table.getTableType())) {
+          if (tableTypes.isEmpty() || tableTypes.contains(
+                tableTypeMapping.mapToClientType(table.getTableType()))) {
             rowSet.addRow(RESULT_SET_SCHEMA, rowData);
           }
         }

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java?rev=1510823&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java Tue Aug  6 00:10:35 2013
@@ -0,0 +1,51 @@
+/**
+ * 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.hive.service.cli.operation;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hive.metastore.TableType;
+
+/**
+ * HiveTableTypeMapping.
+ * Default table type mapping
+ *
+ */
+public class HiveTableTypeMapping implements TableTypeMapping {
+
+  @Override
+  public String mapToHiveType(String clientTypeName) {
+    return clientTypeName;
+  }
+
+  @Override
+  public String mapToClientType(String hiveTypeName) {
+    return hiveTypeName;
+  }
+
+  @Override
+  public Set<String> getTableTypeNames() {
+    Set<String> typeNameSet = new HashSet<String>();
+    for (TableType typeNames : TableType.values()) {
+      typeNameSet.add(typeNames.toString());
+    }
+    return typeNameSet;
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMapping.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMapping.java?rev=1510823&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMapping.java Tue Aug  6 00:10:35 2013
@@ -0,0 +1,44 @@
+/**
+ * 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.hive.service.cli.operation;
+
+import java.util.Set;
+
+
+public interface TableTypeMapping {
+  /**
+   * Map client's table type name to hive's table type
+   * @param clientTypeName
+   * @return
+   */
+  public String mapToHiveType (String clientTypeName);
+
+  /**
+   * Map hive's table type name to client's table type
+   * @param clientTypeName
+   * @return
+   */
+  public String mapToClientType (String hiveTypeName);
+
+  /**
+   * Get all the table types of this mapping
+   * @return
+   */
+  public Set<String> getTableTypeNames();
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java?rev=1510823&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java Tue Aug  6 00:10:35 2013
@@ -0,0 +1,37 @@
+/**
+ * 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.hive.service.cli.operation;
+
+public class TableTypeMappingFactory {
+
+  public enum TableTypeMappings {
+    HIVE,
+    CLASSIC
+  }
+  private static TableTypeMapping hiveTableTypeMapping = new HiveTableTypeMapping();
+  private static TableTypeMapping classicTableTypeMapping = new ClassicTableTypeMapping();
+
+  public static TableTypeMapping getTableTypeMapping(String mappingType) {
+    if (TableTypeMappings.CLASSIC.toString().equalsIgnoreCase(mappingType)) {
+      return classicTableTypeMapping;
+    } else {
+      return hiveTableTypeMapping;
+    }
+  }
+}