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

svn commit: r1528642 - in /hive/branches/branch-0.12/hcatalog: conf/ core/src/main/java/org/apache/hcatalog/security/ core/src/main/java/org/apache/hive/hcatalog/security/ core/src/test/java/org/apache/hcatalog/security/ core/src/test/java/org/apache/h...

Author: thejas
Date: Wed Oct  2 21:51:36 2013
New Revision: 1528642

URL: http://svn.apache.org/r1528642
Log:
HIVE-5413 : StorageDelegationAuthorizationProvider uses non-existent org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler (Eugene Koifman via Sushanth Sowmyan)

Added:
    hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
Removed:
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hive/hcatalog/security/
    hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hive/hcatalog/security/
    hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/HBaseReadWrite.java
Modified:
    hive/branches/branch-0.12/hcatalog/conf/proto-hive-site.xml
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java
    hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java
    hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java
    hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf
    hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/pig.conf
    hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
    hive/branches/branch-0.12/hcatalog/src/test/e2e/templeton/README.txt
    hive/branches/branch-0.12/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java

Modified: hive/branches/branch-0.12/hcatalog/conf/proto-hive-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/conf/proto-hive-site.xml?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/conf/proto-hive-site.xml (original)
+++ hive/branches/branch-0.12/hcatalog/conf/proto-hive-site.xml Wed Oct  2 21:51:36 2013
@@ -114,7 +114,7 @@
 
 <property>
   <name>hive.security.authorization.manager</name>
-  <value>org.apache.hive.hcatalog.security.StorageDelegationAuthorizationProvider</value>
+  <value>org.apache.hcatalog.security.StorageDelegationAuthorizationProvider</value>
   <description>the hive client authorization manager class name.
   The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.
   HCatalog uses a model, where authorization checks are delegated to the storage layer (hdfs, hbase, ...).

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java Wed Oct  2 21:51:36 2013
@@ -56,7 +56,7 @@ import org.apache.hadoop.security.UserGr
  * An AuthorizationProvider, which checks against the data access level permissions on HDFS.
  * It makes sense to eventually move this class to Hive, so that all hive users can
  * use this authorization model. 
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.HdfsAuthorizationProvider} instead
+ * @deprecated use {@link org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider}
  */
 public class HdfsAuthorizationProvider extends HiveAuthorizationProviderBase {
 

Modified: hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java Wed Oct  2 21:51:36 2013
@@ -42,7 +42,7 @@ import org.apache.hcatalog.mapreduce.HCa
 /**
  * A HiveAuthorizationProvider which delegates the authorization requests to 
  * the underlying AuthorizationProviders obtained from the StorageHandler.
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.StorageDelegationAuthorizationProvider} instead
+ * @deprecated 
  */
 public class StorageDelegationAuthorizationProvider extends HiveAuthorizationProviderBase {
 

Modified: hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java (original)
+++ hive/branches/branch-0.12/hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java Wed Oct  2 21:51:36 2013
@@ -53,7 +53,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 /**
- * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.TestHdfsAuthorizationProvider} instead
+ * @deprecated 
  */
 public class TestHdfsAuthorizationProvider {
 
@@ -78,7 +78,7 @@ public class TestHdfsAuthorizationProvid
     conf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
     conf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
     conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
-        StorageDelegationAuthorizationProvider.class.getCanonicalName());
+        StorageDelegationAuthorizationProvider.class.getName());
     conf.set("fs.pfile.impl", "org.apache.hadoop.fs.ProxyLocalFileSystem");
 
     whDir = System.getProperty("test.warehouse.dir", "/tmp/testhdfsauthorization_wh");

Modified: hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf (original)
+++ hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/hadoop.conf Wed Oct  2 21:51:36 2013
@@ -225,9 +225,9 @@ jar :FUNCPATH:/testudf.jar org.apache.hi
                                 {
                                  'num' => 1
                                 ,'hcat_prep'=>q\drop table if exists hadoop_hbase_1;
-create table hadoop_hbase_1(key string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:gpa');\
+create table hadoop_hbase_1(key string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:gpa');\
                                 ,'hadoop' => q\
-jar :FUNCPATH:/testudf.jar org.apache.hive.hcatalog.utils.HBaseReadWrite -libjars :HCAT_JAR: :THRIFTSERVER: :INPATH:/studenttab10k hadoop_hbase_1 :OUTPATH:
+jar :FUNCPATH:/testudf.jar org.apache.hcatalog.utils.HBaseReadWrite -libjars :HCAT_JAR: :THRIFTSERVER: :INPATH:/studenttab10k hadoop_hbase_1 :OUTPATH:
 \,
                                 ,'sql' => q\select name, sum(gpa) from studenttab10k group by name;\
                                 ,'floatpostprocess' => 1

Modified: hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/pig.conf
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/pig.conf?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/pig.conf (original)
+++ hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/tests/pig.conf Wed Oct  2 21:51:36 2013
@@ -319,15 +319,15 @@ store c into ':OUTPATH:';\
                                 {
                                  'num' => 1
                                 ,'hcat_prep'=>q\drop table if exists pig_hbase_1;
-create table pig_hbase_1(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');\
+create table pig_hbase_1(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');\
                                 ,'pig' => q\set hcat.hbase.output.bulkMode 'false'
 a = load ':INPATH:/studenttab10k' as (name:chararray, age:int, gpa:float);
 b = group a by name;
 c = foreach b generate group as name, AVG(a.age) as age, AVG(a.gpa) as gpa;
 d = foreach c generate name as key, (chararray)age, (chararray)gpa as gpa;
-store d into 'pig_hbase_1' using org.apache.hive.hcatalog.pig.HCatStorer();
+store d into 'pig_hbase_1' using org.apache.hcatalog.pig.HCatStorer();
 exec
-e = load 'pig_hbase_1' using org.apache.hive.hcatalog.pig.HCatLoader();
+e = load 'pig_hbase_1' using org.apache.hcatalog.pig.HCatLoader();
 store e into ':OUTPATH:';\,
                                 ,'result_table' => ['pig_hbase_1','?']
 				,'sql'   => [ 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;', 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;' ]
@@ -338,17 +338,17 @@ store e into ':OUTPATH:';\,
                                  # multiquery
                                  'num' => 2
                                 ,'hcat_prep'=>q\drop table if exists pig_hbase_2_1;
-create table pig_hbase_2_1(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
+create table pig_hbase_2_1(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
 drop table if exists pig_hbase_2_2;
-create table pig_hbase_2_2(key string, age string, gpa string) STORED BY 'org.apache.hive.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
+create table pig_hbase_2_2(key string, age string, gpa string) STORED BY 'org.apache.hcatalog.hbase.HBaseHCatStorageHandler' TBLPROPERTIES ('hbase.columns.mapping'=':key,info:age,info:gpa');
 \
                                 ,'pig' => q\set hcat.hbase.output.bulkMode 'false'
 a = load ':INPATH:/studenttab10k' as (name:chararray, age:int, gpa:float);
 b = group a by name;
 c = foreach b generate group as name, AVG(a.age) as age, AVG(a.gpa) as gpa;
 d = foreach c generate name as key, (chararray)age, (chararray)gpa as gpa;
-store d into 'pig_hbase_2_1' using org.apache.hive.hcatalog.pig.HCatStorer();
-store d into 'pig_hbase_2_2' using org.apache.hive.hcatalog.pig.HCatStorer();\,
+store d into 'pig_hbase_2_1' using org.apache.hcatalog.pig.HCatStorer();
+store d into 'pig_hbase_2_2' using org.apache.hcatalog.pig.HCatStorer();\,
                                 ,'result_table' => ['pig_hbase_2_1','pig_hbase_2_2']
 				,'sql'   => [ 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;', 'select name, avg(cast(age as decimal(10,5))), avg(gpa) from studenttab10k group by name;']
                                 ,'floatpostprocess' => 1

Added: hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java?rev=1528642&view=auto
==============================================================================
--- hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java (added)
+++ hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hcatalog/utils/HBaseReadWrite.java Wed Oct  2 21:51:36 2013
@@ -0,0 +1,192 @@
+/**
+ * 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.hcatalog.utils;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hcatalog.common.HCatConstants;
+import org.apache.hcatalog.data.DefaultHCatRecord;
+import org.apache.hcatalog.data.HCatRecord;
+import org.apache.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.hcatalog.mapreduce.HCatOutputFormat;
+import org.apache.hcatalog.mapreduce.InputJobInfo;
+import org.apache.hcatalog.mapreduce.OutputJobInfo;
+
+/**
+ * This is a map reduce test for testing hcat which goes against the "numbers"
+ * table. It performs a group by on the first column and a SUM operation on the
+ * other columns. This is to simulate a typical operation in a map reduce
+ * program to test that hcat hands the right data to the map reduce program
+ *
+ * Usage: hadoop jar sumnumbers <serveruri> <output dir> <-libjars hive-hcat
+ * jar> The <tab|ctrla> argument controls the output delimiter The hcat jar
+ * location should be specified as file://<full path to jar>
+ */
+public class HBaseReadWrite extends Configured implements Tool {
+
+  public static class HBaseWriteMap extends
+    Mapper<LongWritable, Text, Text, Text> {
+
+    String name;
+    String age;
+    String gpa;
+
+    @Override
+    protected void map(
+      LongWritable key,
+      Text value,
+      org.apache.hadoop.mapreduce.Mapper<LongWritable, Text, Text, Text>.Context context)
+      throws IOException, InterruptedException {
+      String line = value.toString();
+      String[] tokens = line.split("\t");
+      name = tokens[0];
+
+      context.write(new Text(name), value);
+    }
+  }
+
+
+  public static class HBaseWriteReduce extends
+    Reducer<Text, Text, WritableComparable, HCatRecord> {
+
+    String name;
+    String age;
+    String gpa;
+
+    @Override
+    protected void reduce(Text key, Iterable<Text> values, Context context)
+      throws IOException, InterruptedException {
+      name = key.toString();
+      int count = 0;
+      double sum = 0;
+      for (Text value : values) {
+        String line = value.toString();
+        String[] tokens = line.split("\t");
+        name = tokens[0];
+        age = tokens[1];
+        gpa = tokens[2];
+
+        count++;
+        sum += Double.parseDouble(gpa.toString());
+      }
+
+      HCatRecord record = new DefaultHCatRecord(2);
+      record.set(0, name);
+      record.set(1, Double.toString(sum));
+
+      context.write(null, record);
+    }
+  }
+
+  public static class HBaseReadMap extends
+    Mapper<WritableComparable, HCatRecord, Text, Text> {
+
+    String name;
+    String age;
+    String gpa;
+
+    @Override
+    protected void map(
+      WritableComparable key,
+      HCatRecord value,
+      org.apache.hadoop.mapreduce.Mapper<WritableComparable, HCatRecord, Text, Text>.Context context)
+      throws IOException, InterruptedException {
+      name = (String) value.get(0);
+      gpa = (String) value.get(1);
+      context.write(new Text(name), new Text(gpa));
+    }
+  }
+
+
+  public int run(String[] args) throws Exception {
+    Configuration conf = getConf();
+    args = new GenericOptionsParser(conf, args).getRemainingArgs();
+
+    String serverUri = args[0];
+    String inputDir = args[1];
+    String tableName = args[2];
+    String outputDir = args[3];
+    String dbName = null;
+
+    String principalID = System
+      .getProperty(HCatConstants.HCAT_METASTORE_PRINCIPAL);
+    if (principalID != null)
+      conf.set(HCatConstants.HCAT_METASTORE_PRINCIPAL, principalID);
+    conf.set("hcat.hbase.output.bulkMode", "false");
+    Job job = new Job(conf, "HBaseWrite");
+    FileInputFormat.setInputPaths(job, inputDir);
+
+    job.setInputFormatClass(TextInputFormat.class);
+    job.setOutputFormatClass(HCatOutputFormat.class);
+    job.setJarByClass(HBaseReadWrite.class);
+    job.setMapperClass(HBaseWriteMap.class);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setReducerClass(HBaseWriteReduce.class);
+    job.setOutputKeyClass(WritableComparable.class);
+    job.setOutputValueClass(DefaultHCatRecord.class);
+    HCatOutputFormat.setOutput(job, OutputJobInfo.create(dbName,
+      tableName, null));
+
+    boolean succ = job.waitForCompletion(true);
+
+    if (!succ) return 1;
+
+    job = new Job(conf, "HBaseRead");
+    HCatInputFormat.setInput(job, InputJobInfo.create(dbName, tableName,
+      null));
+
+    job.setInputFormatClass(HCatInputFormat.class);
+    job.setOutputFormatClass(TextOutputFormat.class);
+    job.setJarByClass(HBaseReadWrite.class);
+    job.setMapperClass(HBaseReadMap.class);
+    job.setOutputKeyClass(Text.class);
+    job.setOutputValueClass(Text.class);
+    job.setNumReduceTasks(0);
+    TextOutputFormat.setOutputPath(job, new Path(outputDir));
+
+    succ = job.waitForCompletion(true);
+
+    if (!succ) return 2;
+
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    int exitCode = ToolRunner.run(new HBaseReadWrite(), args);
+    System.exit(exitCode);
+  }
+}
+

Modified: hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java (original)
+++ hive/branches/branch-0.12/hcatalog/src/test/e2e/hcatalog/udfs/java/org/apache/hive/hcatalog/utils/WriteTextPartitioned.java Wed Oct  2 21:51:36 2013
@@ -48,7 +48,7 @@ import org.apache.hive.hcatalog.mapreduc
  * other columns. This is to simulate a typical operation in a map reduce
  * program to test that hcat hands the right data to the map reduce program
  *
- * Usage: hadoop jar org.apache.hive.hcatalog.utils.HBaseReadWrite -libjars
+ * Usage: hadoop jar org.apache.hcatalog.utils.HBaseReadWrite -libjars
  * &lt;hcat_jar&gt; * &lt;serveruri&gt; &lt;input_tablename&gt; &lt;output_tablename&gt; [filter]
  * If filter is given it will be provided as the partition to write to.
  */

Modified: hive/branches/branch-0.12/hcatalog/src/test/e2e/templeton/README.txt
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/src/test/e2e/templeton/README.txt?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/src/test/e2e/templeton/README.txt (original)
+++ hive/branches/branch-0.12/hcatalog/src/test/e2e/templeton/README.txt Wed Oct  2 21:51:36 2013
@@ -69,6 +69,7 @@ Setup
 2. Install perl and following perl modules  (cpan -i <MODULE_NAME>)
 * IPC::Run
 * JSON
+* JSON::Path
 * Data::Dump
 * Number::Compare
 * Text::Glob

Modified: hive/branches/branch-0.12/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java?rev=1528642&r1=1528641&r2=1528642&view=diff
==============================================================================
--- hive/branches/branch-0.12/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java (original)
+++ hive/branches/branch-0.12/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseAuthorizationProvider.java Wed Oct  2 21:51:36 2013
@@ -22,6 +22,8 @@ package org.apache.hcatalog.hbase;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -34,8 +36,11 @@ import org.apache.hadoop.hive.ql.securit
 /**
  * This class is an implementation of HiveAuthorizationProvider to provide
  * authorization functionality for HBase tables.
+ * @deprecated 
  */
-class HBaseAuthorizationProvider implements HiveAuthorizationProvider {
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HBaseAuthorizationProvider implements HiveAuthorizationProvider {
 
   @Override
   public Configuration getConf() {