You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/02/22 01:20:50 UTC

svn commit: r1570760 [1/7] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ itests/qtest/ itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrif...

Author: hashutosh
Date: Sat Feb 22 00:20:49 2014
New Revision: 1570760

URL: http://svn.apache.org/r1570760
Log:
HIVE-6380 : Specify jars/files when creating permanent UDFs (Jason Dere via Ashutosh Chauhan)

Added:
    hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/UDFFileLookup.java
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceType.java
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ResourceUri.java
    hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MResourceUri.java
    hive/trunk/ql/src/test/queries/clientnegative/udf_local_resource.q
    hive/trunk/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q
    hive/trunk/ql/src/test/queries/clientpositive/udf_using.q
    hive/trunk/ql/src/test/results/clientnegative/udf_local_resource.q.out
    hive/trunk/ql/src/test/results/clientnegative/udf_nonexistent_resource.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf_using.q.out
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
    hive/trunk/itests/qtest/pom.xml
    hive/trunk/metastore/if/hive_metastore.thrift
    hive/trunk/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
    hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
    hive/trunk/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
    hive/trunk/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
    hive/trunk/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
    hive/trunk/metastore/src/gen/thrift/gen-php/metastore/Types.php
    hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
    hive/trunk/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
    hive/trunk/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/trunk/metastore/src/model/org/apache/hadoop/hive/metastore/model/MFunction.java
    hive/trunk/metastore/src/model/package.jdo
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateFunctionDesc.java

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java?rev=1570760&r1=1570759&r2=1570760&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java Sat Feb 22 00:20:49 2014
@@ -56,6 +56,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.ResourceType;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -2505,7 +2507,7 @@ public abstract class TestHiveMetaStore 
 
       createDb(dbName);
 
-      createFunction(dbName, funcName, className, owner, ownerType, createTime, funcType);
+      createFunction(dbName, funcName, className, owner, ownerType, createTime, funcType, null);
 
       // Try the different getters
 
@@ -2517,6 +2519,8 @@ public abstract class TestHiveMetaStore 
       assertEquals("function owner name", owner, func.getOwnerName());
       assertEquals("function owner type", PrincipalType.USER, func.getOwnerType());
       assertEquals("function type", funcType, func.getFunctionType());
+      List<ResourceUri> resources = func.getResourceUris();
+      assertTrue("function resources", resources == null || resources.size() == 0);
 
       boolean gotException = false;
       try {
@@ -2551,7 +2555,52 @@ public abstract class TestHiveMetaStore 
     }
   }
 
+  public void testFunctionWithResources() throws Exception {
+    String dbName = "test_db2";
+    String funcName = "test_func";
+    String className = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper";
+    String owner = "test_owner";
+    PrincipalType ownerType = PrincipalType.USER;
+    int createTime = (int) (System.currentTimeMillis() / 1000);
+    FunctionType funcType = FunctionType.JAVA;
+    List<ResourceUri> resList = new ArrayList<ResourceUri>();
+    resList.add(new ResourceUri(ResourceType.JAR, "hdfs:///tmp/jar1.jar"));
+    resList.add(new ResourceUri(ResourceType.FILE, "hdfs:///tmp/file1.txt"));
+    resList.add(new ResourceUri(ResourceType.ARCHIVE, "hdfs:///tmp/archive1.tgz"));
+
+    try {
+      cleanUp(dbName, null, null);
 
+      createDb(dbName);
+
+      createFunction(dbName, funcName, className, owner, ownerType, createTime, funcType, resList);
+
+      // Try the different getters
+
+      // getFunction()
+      Function func = client.getFunction(dbName, funcName);
+      assertEquals("function db name", dbName, func.getDbName());
+      assertEquals("function name", funcName, func.getFunctionName());
+      assertEquals("function class name", className, func.getClassName());
+      assertEquals("function owner name", owner, func.getOwnerName());
+      assertEquals("function owner type", PrincipalType.USER, func.getOwnerType());
+      assertEquals("function type", funcType, func.getFunctionType());
+      List<ResourceUri> resources = func.getResourceUris();
+      assertEquals("Resource list size", resList.size(), resources.size());
+      for (ResourceUri res : resources) {
+        assertTrue("Matching resource " + res.getResourceType() + " " + res.getUri(),
+            resList.indexOf(res) >= 0);
+      }
+
+      client.dropFunction(dbName, funcName);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testConcurrentMetastores() failed.");
+      throw e;
+    } finally {
+      silentDropDatabase(dbName);
+    }
+  }
 
   /**
    * This method simulates another Hive metastore renaming a table, by accessing the db and
@@ -2737,9 +2786,10 @@ public abstract class TestHiveMetaStore 
 
   private void createFunction(String dbName, String funcName, String className,
       String ownerName, PrincipalType ownerType, int createTime,
-      org.apache.hadoop.hive.metastore.api.FunctionType functionType) throws Exception {
+      org.apache.hadoop.hive.metastore.api.FunctionType functionType, List<ResourceUri> resources)
+          throws Exception {
     Function func = new Function(funcName, dbName, className,
-        ownerName, ownerType, createTime, functionType);
+        ownerName, ownerType, createTime, functionType, resources);
     client.createFunction(func);
   }
 }

Modified: hive/trunk/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/itests/qtest/pom.xml?rev=1570760&r1=1570759&r2=1570760&view=diff
==============================================================================
--- hive/trunk/itests/qtest/pom.xml (original)
+++ hive/trunk/itests/qtest/pom.xml Sat Feb 22 00:20:49 2014
@@ -36,8 +36,8 @@
     <run_disabled>false</run_disabled>
     <clustermode></clustermode>
     <execute.beeline.tests>false</execute.beeline.tests>
-    <minimr.query.files>stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q</minimr.query.files>
-    <minimr.query.negative.files>cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q</minimr.query.negative.files>
+    <minimr.query.files>stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q,udf_using.q</minimr.query.files>
+    <minimr.query.negative.files>cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q,udf_local_resource.q</minimr.query.negative.files>
     <minitez.query.files>tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q</minitez.query.files>
     <minitez.query.files.shared>join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q,stats_counter.q,stats_noscan_1.q,stats_counter_partitioned.q</minitez.query.files.shared>
     <beeline.positive.exclude>add_part_exist.q,alter1.q,alter2.q,alter4.q,alter5.q,alter_rename_partition.q,alter_rename_partition_authorization.q,archive.q,archive_corrupt.q,archive_multi.q,archive_mr_1806.q,archive_multi_mr_1806.q,authorization_1.q,authorization_2.q,authorization_4.q,authorization_5.q,authorization_6.q,authorization_7.q,ba_table1.q,ba_table2.q,ba_table3.q,ba_table_udfs.q,binary_table_bincolserde.q,binary_table_colserde.q,cluster.q,columnarserde_create_shortcut.q,combine2.q,constant_prop.q,create_nested_type.q,create_or_replace_view.q,create_struct_table.q,create_union_table.q,database.q,database_location.q,database_properties.q,ddltime.q,describe_database_json.q,drop_database_removes_partition_dirs.q,escape1.q,escape2.q,exim_00_nonpart_empty.q,exim_01_nonpart.q,exim_02_00_part_empty.q,exim_02_part.q,exim_03_nonpart_over_compat.q,exim_04_all_part.q,exim_04_evolved_parts.q,exim_05_some_part.q,exim_06_one_part.q,exim_07_all_part_over_nonoverlap.q,exim_08_nonpart_rena
 me.q,exim_09_part_spec_nonoverlap.q,exim_10_external_managed.q,exim_11_managed_external.q,exim_12_external_location.q,exim_13_managed_location.q,exim_14_managed_location_over_existing.q,exim_15_external_part.q,exim_16_part_external.q,exim_17_part_managed.q,exim_18_part_external.q,exim_19_00_part_external_location.q,exim_19_part_external_location.q,exim_20_part_managed_location.q,exim_21_export_authsuccess.q,exim_22_import_exist_authsuccess.q,exim_23_import_part_authsuccess.q,exim_24_import_nonexist_authsuccess.q,global_limit.q,groupby_complex_types.q,groupby_complex_types_multi_single_reducer.q,index_auth.q,index_auto.q,index_auto_empty.q,index_bitmap.q,index_bitmap1.q,index_bitmap2.q,index_bitmap3.q,index_bitmap_auto.q,index_bitmap_rc.q,index_compact.q,index_compact_1.q,index_compact_2.q,index_compact_3.q,index_stale_partitioned.q,init_file.q,input16.q,input16_cc.q,input46.q,input_columnarserde.q,input_dynamicserde.q,input_lazyserde.q,input_testxpath3.q,input_testxpath4.q,insert2_o
 verwrite_partitions.q,insertexternal1.q,join_thrift.q,lateral_view.q,load_binary_data.q,load_exist_part_authsuccess.q,load_nonpart_authsuccess.q,load_part_authsuccess.q,loadpart_err.q,lock1.q,lock2.q,lock3.q,lock4.q,merge_dynamic_partition.q,multi_insert.q,multi_insert_move_tasks_share_dependencies.q,null_column.q,ppd_clusterby.q,query_with_semi.q,rename_column.q,sample6.q,sample_islocalmode_hook.q,set_processor_namespaces.q,show_tables.q,source.q,split_sample.q,str_to_map.q,transform1.q,udaf_collect_set.q,udaf_context_ngrams.q,udaf_histogram_numeric.q,udaf_ngrams.q,udaf_percentile_approx.q,udf_array.q,udf_bitmap_and.q,udf_bitmap_or.q,udf_explode.q,udf_format_number.q,udf_map.q,udf_map_keys.q,udf_map_values.q,udf_max.q,udf_min.q,udf_named_struct.q,udf_percentile.q,udf_printf.q,udf_sentences.q,udf_sort_array.q,udf_split.q,udf_struct.q,udf_substr.q,udf_translate.q,udf_union.q,udf_xpath.q,udtf_stack.q,view.q,virtual_column.q</beeline.positive.exclude>

Added: hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/UDFFileLookup.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/UDFFileLookup.java?rev=1570760&view=auto
==============================================================================
--- hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/UDFFileLookup.java (added)
+++ hive/trunk/itests/util/src/main/java/org/apache/hadoop/hive/ql/udf/UDFFileLookup.java Sat Feb 22 00:20:49 2014
@@ -0,0 +1,71 @@
+/**
+ * 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.udf;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * A UDF for testing, which does key/value lookup from a file
+ */
+public class UDFFileLookup extends UDF {
+  static Log LOG = LogFactory.getLog(UDFFileLookup.class);
+
+  IntWritable result = new IntWritable();
+
+  static Map<String, Integer> data = null;
+
+  private static void loadData() throws Exception {
+    // sales.txt will need to be added as a resource.
+    File file = new File("./sales.txt");
+    BufferedReader br = new BufferedReader(new FileReader(file));
+    data = new HashMap<String, Integer>();
+    String line = br.readLine();
+    while (line != null) {
+      String[] parts = line.split("\t", 2);
+      if (parts.length == 2) {
+        data.put(parts[0], Integer.valueOf(parts[1]));
+      }
+      line = br.readLine();
+    }
+    br.close();
+  }
+
+  public IntWritable evaluate(Text s) throws Exception {
+    if (data == null) {
+      loadData();
+    }
+    Integer val = data.get(s.toString());
+    if (val == null) {
+      return null;
+    }
+    result.set(val.intValue());
+    return result;
+  }
+}

Modified: hive/trunk/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/if/hive_metastore.thrift?rev=1570760&r1=1570759&r2=1570760&view=diff
==============================================================================
--- hive/trunk/metastore/if/hive_metastore.thrift (original)
+++ hive/trunk/metastore/if/hive_metastore.thrift Sat Feb 22 00:20:49 2014
@@ -360,6 +360,17 @@ enum FunctionType {
   JAVA = 1,
 }
 
+enum ResourceType {
+  JAR     = 1,
+  FILE    = 2,
+  ARCHIVE = 3,
+}
+
+struct ResourceUri {
+  1: ResourceType resourceType,
+  2: string       uri,
+}
+
 // User-defined function
 struct Function {
   1: string           functionName,
@@ -369,6 +380,7 @@ struct Function {
   5: PrincipalType    ownerType,
   6: i32              createTime,
   7: FunctionType     functionType,
+  8: list<ResourceUri> resourceUris,
 }
 
 exception MetaException {