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

svn commit: r1520467 - in /hive/trunk/hcatalog: core/src/main/java/org/apache/hcatalog/data/transfer/impl/ core/src/main/java/org/apache/hcatalog/mapreduce/ core/src/main/java/org/apache/hcatalog/shims/ core/src/test/java/org/apache/hcatalog/cli/ core/...

Author: khorgath
Date: Fri Sep  6 00:52:33 2013
New Revision: 1520467

URL: http://svn.apache.org/r1520467
Log:
HIVE-4896 : create binary backwards compatibility layer hcatalog 0.12 and 0.11 - Part 2/4 - fix 0.11 code to use new shim layer (Eugene Koifman via Sushanth Sowmyan)

Removed:
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/shims/HCatHadoopShims.java
Modified:
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java
    hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java Fri Sep  6 00:52:33 2013
@@ -38,7 +38,7 @@ import org.apache.hcatalog.data.transfer
 import org.apache.hcatalog.data.transfer.ReaderContext;
 import org.apache.hcatalog.data.transfer.state.StateProvider;
 import org.apache.hcatalog.mapreduce.HCatInputFormat;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 /**
  * This reader reads via {@link HCatInputFormat}
@@ -66,7 +66,7 @@ public class HCatInputFormatReader exten
                 job, re.getDbName(), re.getTableName()).setFilter(re.getFilterString());
             ReaderContext cntxt = new ReaderContext();
             cntxt.setInputSplits(hcif.getSplits(
-                HCatHadoopShims.Instance.get().createJobContext(job.getConfiguration(), null)));
+                ShimLoader.getHadoopShims().getHCatShim().createJobContext(job.getConfiguration(), null)));
             cntxt.setConf(job.getConfiguration());
             return cntxt;
         } catch (IOException e) {
@@ -82,7 +82,7 @@ public class HCatInputFormatReader exten
         HCatInputFormat inpFmt = new HCatInputFormat();
         RecordReader<WritableComparable, HCatRecord> rr;
         try {
-            TaskAttemptContext cntxt = HCatHadoopShims.Instance.get().createTaskAttemptContext(conf, new TaskAttemptID());
+            TaskAttemptContext cntxt = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(conf, new TaskAttemptID());
             rr = inpFmt.createRecordReader(split, cntxt);
             rr.initialize(split, cntxt);
         } catch (IOException e) {

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java Fri Sep  6 00:52:33 2013
@@ -40,7 +40,7 @@ import org.apache.hcatalog.data.transfer
 import org.apache.hcatalog.data.transfer.state.StateProvider;
 import org.apache.hcatalog.mapreduce.HCatOutputFormat;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 /**
  * This writer writes via {@link HCatOutputFormat}
@@ -67,8 +67,8 @@ public class HCatOutputFormatWriter exte
             HCatOutputFormat.setSchema(job, HCatOutputFormat.getTableSchema(job));
             HCatOutputFormat outFormat = new HCatOutputFormat();
             outFormat.checkOutputSpecs(job);
-            outFormat.getOutputCommitter(HCatHadoopShims.Instance.get().createTaskAttemptContext
-                (job.getConfiguration(), HCatHadoopShims.Instance.get().createTaskAttemptID())).setupJob(job);
+            outFormat.getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(
+                job.getConfiguration(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID())).setupJob(job);
         } catch (IOException e) {
             throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e);
         } catch (InterruptedException e) {
@@ -85,8 +85,8 @@ public class HCatOutputFormatWriter exte
         int id = sp.getId();
         setVarsInConf(id);
         HCatOutputFormat outFormat = new HCatOutputFormat();
-        TaskAttemptContext cntxt = HCatHadoopShims.Instance.get().createTaskAttemptContext
-            (conf, new TaskAttemptID(HCatHadoopShims.Instance.get().createTaskID(), id));
+        TaskAttemptContext cntxt = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(
+            conf, new TaskAttemptID(ShimLoader.getHadoopShims().getHCatShim().createTaskID(), id));
         OutputCommitter committer = null;
         RecordWriter<WritableComparable<?>, HCatRecord> writer;
         try {
@@ -125,9 +125,9 @@ public class HCatOutputFormatWriter exte
     @Override
     public void commit(WriterContext context) throws HCatException {
         try {
-            new HCatOutputFormat().getOutputCommitter(HCatHadoopShims.Instance.get().createTaskAttemptContext
-                (context.getConf(), HCatHadoopShims.Instance.get().createTaskAttemptID()))
-                .commitJob(HCatHadoopShims.Instance.get().createJobContext(context.getConf(), null));
+            new HCatOutputFormat().getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(
+                context.getConf(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID()))
+                .commitJob(ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConf(), null));
         } catch (IOException e) {
             throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e);
         } catch (InterruptedException e) {
@@ -138,9 +138,9 @@ public class HCatOutputFormatWriter exte
     @Override
     public void abort(WriterContext context) throws HCatException {
         try {
-            new HCatOutputFormat().getOutputCommitter(HCatHadoopShims.Instance.get().createTaskAttemptContext
-                (context.getConf(), HCatHadoopShims.Instance.get().createTaskAttemptID()))
-                .abortJob(HCatHadoopShims.Instance.get().createJobContext(context.getConf(), null), State.FAILED);
+            new HCatOutputFormat().getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(
+                context.getConf(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID()))
+                .abortJob(ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConf(), null), State.FAILED);
         } catch (IOException e) {
             throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e);
         } catch (InterruptedException e) {

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java Fri Sep  6 00:52:33 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.mapred.HCatMapRedUtil;
+import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java Fri Sep  6 00:52:33 2013
@@ -43,7 +43,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.mapred.HCatMapRedUtil;
+import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobStatus.State;
@@ -56,7 +56,7 @@ import org.apache.hcatalog.data.schema.H
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.data.schema.HCatSchemaUtils;
 import org.apache.hcatalog.har.HarOutputCommitterPostProcessor;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -304,7 +304,7 @@ class FileOutputCommitterContainer exten
 
         // Apply the group and permissions to the leaf partition and files.
         // Need not bother in case of HDFS as permission is taken care of by setting UMask
-        if (!HCatHadoopShims.Instance.get().isFileInHDFS(fs, partPath)) {
+        if (!ShimLoader.getHadoopShims().getHCatShim().isFileInHDFS(fs, partPath)) {
             applyGroupAndPerms(fs, partPath, perms, grpName, true);
         }
 
@@ -578,7 +578,7 @@ class FileOutputCommitterContainer exten
                         jobConf,
                         context.getJobID(),
                         InternalUtil.createReporter(HCatMapRedUtil.createTaskAttemptContext(jobConf,
-                            HCatHadoopShims.Instance.get().createTaskAttemptID())));
+                            ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID())));
                     HCatOutputFormat.configureOutputStorageHandler(currContext, jobInfo, fullPartSpec);
                     contextDiscoveredByPath.put(st.getPath().toString(), currContext);
                 }

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java Fri Sep  6 00:52:33 2013
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapred.HCatMapRedUtil;
+import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.OutputCommitter;

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java Fri Sep  6 00:52:33 2013
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hcatalog.common.HCatConstants;
 import org.apache.hcatalog.common.HCatUtil;
@@ -180,5 +181,13 @@ public class FosterStorageHandler extend
         throws HiveException {
         return new DefaultHiveAuthorizationProvider();
     }
+    @Override
+    public void configureJobConf(TableDesc tableDesc, JobConf jobConf) {
+        //do nothing by default
+        //EK: added the same (no-op) implementation as in 
+        // org.apache.hive.hcatalog.DefaultStorageHandler (hive 0.12)
+        // this is needed to get 0.11 API compat layer to work
+        // see HIVE-4896
+    }
 
 }

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java Fri Sep  6 00:52:33 2013
@@ -36,6 +36,7 @@ import org.apache.commons.lang.StringUti
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Job;
@@ -48,7 +49,7 @@ import org.apache.hadoop.mapreduce.TaskA
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hcatalog.common.HCatUtil;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -146,13 +147,13 @@ public class MultiOutputFormat extends O
 
     static {
         configsToOverride.add("mapred.output.dir");
-        configsToOverride.add(HCatHadoopShims.Instance.get().getPropertyName(HCatHadoopShims.PropertyName.CACHE_SYMLINK));
+        configsToOverride.add(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_SYMLINK));
         configsToMerge.put(JobContext.JOB_NAMENODES, COMMA_DELIM);
         configsToMerge.put("tmpfiles", COMMA_DELIM);
         configsToMerge.put("tmpjars", COMMA_DELIM);
         configsToMerge.put("tmparchives", COMMA_DELIM);
-        configsToMerge.put(HCatHadoopShims.Instance.get().getPropertyName(HCatHadoopShims.PropertyName.CACHE_ARCHIVES), COMMA_DELIM);
-        configsToMerge.put(HCatHadoopShims.Instance.get().getPropertyName(HCatHadoopShims.PropertyName.CACHE_FILES), COMMA_DELIM);
+        configsToMerge.put(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_ARCHIVES), COMMA_DELIM);
+        configsToMerge.put(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_FILES), COMMA_DELIM);
         String fileSep;
         if (HCatUtil.isHadoop23()) {
             fileSep = ",";
@@ -183,7 +184,7 @@ public class MultiOutputFormat extends O
      */
     public static JobContext getJobContext(String alias, JobContext context) {
         String aliasConf = context.getConfiguration().get(getAliasConfName(alias));
-        JobContext aliasContext = HCatHadoopShims.Instance.get().createJobContext(context.getConfiguration(), context.getJobID());
+        JobContext aliasContext = ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConfiguration(), context.getJobID());
         addToConfig(aliasConf, aliasContext.getConfiguration());
         return aliasContext;
     }
@@ -197,7 +198,7 @@ public class MultiOutputFormat extends O
      */
     public static TaskAttemptContext getTaskAttemptContext(String alias, TaskAttemptContext context) {
         String aliasConf = context.getConfiguration().get(getAliasConfName(alias));
-        TaskAttemptContext aliasContext = HCatHadoopShims.Instance.get().createTaskAttemptContext(
+        TaskAttemptContext aliasContext = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(
                 context.getConfiguration(), context.getTaskAttemptID());
         addToConfig(aliasConf, aliasContext.getConfiguration());
         return aliasContext;

Modified: hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java (original)
+++ hive/trunk/hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java Fri Sep  6 00:52:33 2013
@@ -39,7 +39,7 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hcatalog.common.HCatConstants;
 import org.apache.hcatalog.common.HCatUtil;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -142,7 +142,7 @@ final class Security {
                     TokenSelector<? extends TokenIdentifier> jtTokenSelector =
                         new org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenSelector();
                     Token jtToken = jtTokenSelector.selectToken(org.apache.hadoop.security.SecurityUtil.buildTokenService(
-                        HCatHadoopShims.Instance.get().getResourceManagerAddress(conf)), ugi.getTokens());
+                        ShimLoader.getHadoopShims().getHCatShim().getResourceManagerAddress(conf)), ugi.getTokens());
                     if (jtToken == null) {
                         //we don't need to cancel this token as the TokenRenewer for JT tokens
                         //takes care of cancelling them

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java Fri Sep  6 00:52:33 2013
@@ -95,6 +95,14 @@ class DummyStorageHandler extends HCatSt
         throws HiveException {
         return new DummyAuthProvider();
     }
+    @Override
+    public void configureJobConf(TableDesc tableDesc, JobConf jobConf) {
+        //do nothing by default
+        //EK: added the same (no-op) implementation as in
+        // org.apache.hive.hcatalog.DefaultStorageHandler (hive 0.12)
+        // this is needed to get 0.11 API compat layer to work
+        // see HIVE-4896
+    }
 
     private class DummyAuthProvider implements HiveAuthorizationProvider {
 

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java Fri Sep  6 00:52:33 2013
@@ -207,7 +207,7 @@ public class TestSemanticAnalysis extend
     public void testAddReplaceCols() throws IOException, MetaException, TException, NoSuchObjectException, CommandNeedRetryException {
 
         hcatDriver.run("drop table junit_sem_analysis");
-        hcatDriver.run("create table junit_sem_analysis (a int, c string) partitioned by (b string) stored as RCFILE");
+        hcatDriver.run("create table junit_sem_analysis (a int, c string) partitioned by (b string)");
         CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis replace columns (a1 tinyint)");
         assertEquals(0, response.getResponseCode());
 

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken Fri Sep  6 00:52:33 2013
@@ -50,14 +50,14 @@ import org.apache.hcatalog.data.HCatData
 import org.apache.hcatalog.data.HCatRecord;
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.rcfile.RCFileInputDriver;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 
 public class TestRCFileInputStorageDriver extends TestCase{
   private static final Configuration conf = new Configuration();
   private static final Path dir =  new Path(System.getProperty("test.data.dir", ".") + "/mapred");
   private static final Path file = new Path(dir, "test_rcfile");
-  private final HCatHadoopShims shim = HCatHadoopShims.Instance.get();
+  private final HadoopShims.HCatHadoopShims shim = ShimLoader.getHadoopShims().getHCatShim();
 
   // Generate sample records to compare against
   private byte[][][] getRecords() throws UnsupportedEncodingException {

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java Fri Sep  6 00:52:33 2013
@@ -43,7 +43,7 @@ import org.apache.hadoop.mapreduce.JobCo
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -233,7 +233,7 @@ public class TestRCFileMapReduceInputFor
         assertEquals("splits length should be " + splitNumber, splits.size(), splitNumber);
         int readCount = 0;
         for (int i = 0; i < splits.size(); i++) {
-            TaskAttemptContext tac = HCatHadoopShims.Instance.get().createTaskAttemptContext(jonconf, new TaskAttemptID());
+            TaskAttemptContext tac = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(jonconf, new TaskAttemptID());
             RecordReader<LongWritable, BytesRefArrayWritable> rr = inputFormat.createRecordReader(splits.get(i), tac);
             rr.initialize(splits.get(i), tac);
             while (rr.nextKeyValue()) {

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken Fri Sep  6 00:52:33 2013
@@ -41,13 +41,13 @@ import org.apache.hcatalog.data.schema.H
 import org.apache.hcatalog.mapreduce.HCatInputStorageDriver;
 import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 public class TestRCFileOutputStorageDriver extends TestCase {
 
   public void testConversion() throws IOException {
     Configuration conf = new Configuration();
-    JobContext jc = HCatHadoopShims.Instance.get().createJobContext(conf, new JobID());
+    JobContext jc = ShimLoader.getHadoopShims().getHCatShim().createJobContext(conf, new JobID());
     String jobString = HCatUtil.serialize(OutputJobInfo.create(null,null,null));
     jc.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO,jobString);
 

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken Fri Sep  6 00:52:33 2013
@@ -50,14 +50,14 @@ import org.apache.hcatalog.data.HCatData
 import org.apache.hcatalog.data.HCatRecord;
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.rcfile.RCFileInputDriver;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 
 public class TestRCFileInputStorageDriver extends TestCase{
   private static final Configuration conf = new Configuration();
   private static final Path dir =  new Path(System.getProperty("test.data.dir", ".") + "/mapred");
   private static final Path file = new Path(dir, "test_rcfile");
-  private final HCatHadoopShims shim = HCatHadoopShims.Instance.get();
+  private final HCatHadoopShims shim = ShimLoader.getHadoopShims().getHCatShim();
 
   // Generate sample records to compare against
   private byte[][][] getRecords() throws UnsupportedEncodingException {

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken Fri Sep  6 00:52:33 2013
@@ -41,13 +41,13 @@ import org.apache.hcatalog.data.schema.H
 import org.apache.hcatalog.mapreduce.HCatInputStorageDriver;
 import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 public class TestRCFileOutputStorageDriver extends TestCase {
 
   public void testConversion() throws IOException {
     Configuration conf = new Configuration();
-    JobContext jc = HCatHadoopShims.Instance.get().createJobContext(conf, new JobID());
+    JobContext jc = ShimLoader.getHadoopShims().getHCatShim().createJobContext(conf, new JobID());
     String jobString = HCatUtil.serialize(OutputJobInfo.create(null,null,null));
     jc.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO,jobString);
 

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java?rev=1520467&r1=1520466&r2=1520467&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java Fri Sep  6 00:52:33 2013
@@ -36,7 +36,7 @@ import org.apache.hcatalog.common.HCatEx
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.mapreduce.HCatOutputFormat;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
-import org.apache.hcatalog.shims.HCatHadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.pig.PigException;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.impl.logicalLayer.FrontendException;
@@ -157,11 +157,11 @@ public class HCatStorer extends HCatBase
 
     @Override
     public void storeSchema(ResourceSchema schema, String arg1, Job job) throws IOException {
-        HCatHadoopShims.Instance.get().commitJob(getOutputFormat(), job);
+        ShimLoader.getHadoopShims().getHCatShim().commitJob(getOutputFormat(), job);
     }
 
     @Override
     public void cleanupOnFailure(String location, Job job) throws IOException {
-        HCatHadoopShims.Instance.get().abortJob(getOutputFormat(), job);
+        ShimLoader.getHadoopShims().getHCatShim().abortJob(getOutputFormat(), job);
     }
 }