incubator-hcatalog-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From khorg...@apache.org
Subject svn commit: r1231282 - in /incubator/hcatalog/trunk: ./ shims/ shims/src/ shims/src/20S/ shims/src/20S/java/ shims/src/20S/java/org/ shims/src/20S/java/org/apache/ shims/src/20S/java/org/apache/hcatalog/ shims/src/20S/java/org/apache/hcatalog/shims/ sh...
Date Fri, 13 Jan 2012 20:10:16 GMT
Author: khorgath
Date: Fri Jan 13 20:10:15 2012
New Revision: 1231282

URL: http://svn.apache.org/viewvc?rev=1231282&view=rev
Log:
HCATALOG-179 Make HCatalog compile against Hadoop 0.23 (thw via khorgath)

Added:
    incubator/hcatalog/trunk/shims/
    incubator/hcatalog/trunk/shims/src/
    incubator/hcatalog/trunk/shims/src/20S/
    incubator/hcatalog/trunk/shims/src/20S/java/
    incubator/hcatalog/trunk/shims/src/20S/java/org/
    incubator/hcatalog/trunk/shims/src/20S/java/org/apache/
    incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/
    incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/
    incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/HCatHadoopShims20S.java
    incubator/hcatalog/trunk/shims/src/23/
    incubator/hcatalog/trunk/shims/src/23/java/
    incubator/hcatalog/trunk/shims/src/23/java/org/
    incubator/hcatalog/trunk/shims/src/23/java/org/apache/
    incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/
    incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/
    incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/HCatHadoopShims23.java
Modified:
    incubator/hcatalog/trunk/CHANGES.txt
    incubator/hcatalog/trunk/build-common.xml
    incubator/hcatalog/trunk/build.xml
    incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
    incubator/hcatalog/trunk/src/java/org/apache/hcatalog/pig/HCatStorer.java
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
    incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java

Modified: incubator/hcatalog/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/CHANGES.txt?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/CHANGES.txt (original)
+++ incubator/hcatalog/trunk/CHANGES.txt Fri Jan 13 20:10:15 2012
@@ -52,6 +52,8 @@ Trunk (unreleased changes)
   HCAT-63. RPM package integration with Hadoop (khorgath via hashutosh)
 
   IMPROVEMENTS
+  HCAT-179. Make HCatalog compile against Hadoop 0.23 (thw via khorgath)
+
   HCAT-194. Better error messages for HCatalog access control errors (julienledem via hashutosh)
 
 
   HCAT-184. Optionally do not generate forrest docs (traviscrawford via hashutosh)

Modified: incubator/hcatalog/trunk/build-common.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/build-common.xml?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/build-common.xml (original)
+++ incubator/hcatalog/trunk/build-common.xml Fri Jan 13 20:10:15 2012
@@ -11,6 +11,22 @@
 
     <!-- hive -->
     <property name="hive.root" value="${common.basedir}/hive/external"/>
+    <property file="${hive.root}/build.properties" prefix="hive."/>
+    <property name="shims.name" value="20S" />
+    <property name="shims.20S.hive.shims.include" value="0.20,0.20S" />
+    <property name="shims.20S.hadoop.version" value="${hive.hadoop-0.20S.version}" />
+    <property name="shims.23.hive.shims.include" value="0.23" />
+    <property name="shims.23.hadoop.version" value="${hive.hadoop-0.23.version}" />
+
+    <!-- macro to accomplish nested expansion like ${p1.${p2}.name} -->
+    <macrodef name="expandToProperty">
+     <attribute name="name"/>
+     <attribute name="value"/>
+     <sequential>
+        <property name="@{name}" value="${@{value}}" />
+     </sequential>
+    </macrodef>
+    <expandToProperty name="hadoop.version" value="shims.${shims.name}.hadoop.version"/>
 
     <!-- common classpaths for various builds -->
     <path id="common.classpath">
@@ -22,8 +38,14 @@
         <fileset dir="${hive.root}/build/serde" includes="*.jar"/>
         <fileset dir="${hive.root}/build/metastore" includes="*.jar"/>
         <fileset dir="${hive.root}/build/ql" includes="*.jar"/>
-        <fileset dir="${hive.root}/build/hadoopcore/hadoop-0.20.3-CDH3-SNAPSHOT/"
-                 includes="hadoop-core-0.20.3-CDH3-SNAPSHOT.jar"/>
+        <fileset dir="${hive.root}/build/hadoopcore/hadoop-${hadoop.version}/">
+          <include name="**/hadoop-*.jar" />
+          <exclude name="**/*test*.jar" />
+          <!-- below is for 0.23 onwards -->
+          <!--include name="share/hadoop/common/lib/*.jar" /-->
+          <exclude name="share/hadoop/common/lib/hadoop-mapreduce-*.jar" />
+          <exclude name="share/hadoop/common/lib/hadoop-yarn-*.jar" />
+        </fileset>
         <fileset dir="${common.ivy.lib.dir}" includes="*.jar"/>
         <fileset dir="${hive.root}/build/ivy/lib/default" includes="antlr-3.0.1.jar"/>
         <fileset dir="${hive.root}/build/ivy/lib/default" includes="commons-lang-*.jar"/>
@@ -35,5 +57,4 @@
         <fileset dir="${hive.root}/build/ivy/lib/default" includes="libfb303-*.jar"/>
         <fileset dir="${hive.root}/lib" includes="asm-3.1.jar"/>
     </path>
-
 </project>

Modified: incubator/hcatalog/trunk/build.xml
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/build.xml?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/build.xml (original)
+++ incubator/hcatalog/trunk/build.xml Fri Jan 13 20:10:15 2012
@@ -125,23 +125,17 @@
       <include name="**/*.jar" />
     </fileset>
     <!-- jars Hadoop depends on -->
-    <fileset dir="${hive.root}/build/hadoopcore/hadoop-0.20.3-CDH3-SNAPSHOT/lib/" >
-      <include name="**/*.jar" />
-    </fileset>
-    <!--
-    <pathelement location="${test.src.data.dir}/conf"/>
-    <pathelement location="${hadoop.oldstyle-name.test.jar}"/>
-    <pathelement location="${hadoop.newstyle-name.test.jar}"/>
-    <pathelement location="${common.jar}"/>
-    <pathelement location="${jsp.test.jar}"/>
-    -->
     <pathelement location="${hcatalog.jar}"/>
-    <!--
-    <pathelement location="${hadoop.root}/lib/jsp-2.1/jsp-api-2.1.jar"/>
-    -->
     <path refid="classpath"/>
-    <fileset dir="${hive.root}/build/hadoopcore/hadoop-0.20.3-CDH3-SNAPSHOT/"
-      includes="hadoop-test-0.20.3-CDH3-SNAPSHOT.jar"/>
+    <fileset dir="${hive.root}/build/hadoopcore/hadoop-${hadoop.version}/">
+      <include name="**/hadoop-*.jar" />
+      <include name="lib/**/*.jar" />
+      <exclude name="lib/**/excluded/" />
+      <!-- below is for 0.23 onwards -->
+      <include name="share/hadoop/common/lib/*.jar" />
+      <exclude name="share/hadoop/common/lib/hadoop-mapreduce-*.jar" />
+      <exclude name="share/hadoop/common/lib/hadoop-yarn-*.jar" />
+    </fileset>
   </path>
 
   <!--
@@ -226,7 +220,13 @@
 
   <!-- Build the external hive code -->
   <target name="hive.jar">
-    <ant antfile="build.xml" dir="${hive.root}" target="package" useNativeBasedir='true'/>
+    <echo message="Building hive with hadoop.version ${hadoop.version}" />
+    <local name="param.shims.include"/>
+    <expandToProperty name="param.shims.include" value="shims.${shims.name}.hive.shims.include"/>
+    <ant antfile="build.xml" dir="${hive.root}" target="package" useNativeBasedir='true'>
+       <property name="shims.include" value="${param.shims.include}"/>
+       <property name="hadoop.version" value="${hadoop.version}"/>
+    </ant>
   </target>
 
   <!--
@@ -244,6 +244,17 @@
       <compilerarg line="${javac.args}"/>
       <classpath refid="classpath" />
     </javac>
+    <!-- compile shim for selected hadoop version -->
+    <!--property name="debugclasspath" refid="classpath"/>
+    <echo message="classpath = ${debugclasspath}"/-->
+    <javac encoding="${build.encoding}" srcdir="${basedir}/shims/src/${shims.name}/java"
excludes="${excludes}"
+        includes="**/*.java" destdir="${build.classes}" debug="${javac.debug}"
+        optimize="${javac.optimize}" target="${javac.version}"
+        source="${javac.version}" deprecation="${javac.deprecation}"
+        includeantruntime="false">
+      <compilerarg line="${javac.args}"/>
+      <classpath refid="classpath" />
+    </javac>
   </target>
  
   <!-- Build the hcatalog client jar -->

Added: incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/HCatHadoopShims20S.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/HCatHadoopShims20S.java?rev=1231282&view=auto
==============================================================================
--- incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/HCatHadoopShims20S.java
(added)
+++ incubator/hcatalog/trunk/shims/src/20S/java/org/apache/hcatalog/shims/HCatHadoopShims20S.java
Fri Jan 13 20:10:15 2012
@@ -0,0 +1,39 @@
+/*
+ * 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.shims;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+public class HCatHadoopShims20S implements HCatHadoopShims {
+
+	@Override
+	public TaskAttemptContext createTaskAttemptContext(Configuration conf,
+			TaskAttemptID taskId) {
+        return new TaskAttemptContext(conf, taskId);
+    }
+
+    @Override
+    public JobContext createJobContext(Configuration conf,
+            JobID jobId) {
+        return new JobContext(conf, jobId);
+    }
+}

Added: incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/HCatHadoopShims23.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/HCatHadoopShims23.java?rev=1231282&view=auto
==============================================================================
--- incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/HCatHadoopShims23.java
(added)
+++ incubator/hcatalog/trunk/shims/src/23/java/org/apache/hcatalog/shims/HCatHadoopShims23.java
Fri Jan 13 20:10:15 2012
@@ -0,0 +1,43 @@
+/*
+ * 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.shims;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+public class HCatHadoopShims23 implements HCatHadoopShims {
+
+	@Override
+	public TaskAttemptContext createTaskAttemptContext(Configuration conf,
+			TaskAttemptID taskId) {
+        return new TaskAttemptContextImpl(conf, taskId);
+	}
+
+	@Override
+    public JobContext createJobContext(Configuration conf,
+            JobID jobId) {
+        JobContext newContext = new JobContextImpl(conf, jobId);
+        return newContext;
+    }
+
+}

Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
(original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java
Fri Jan 13 20:10:15 2012
@@ -47,6 +47,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.thrift.TException;
 
 import java.io.IOException;
@@ -146,7 +147,7 @@ class FileOutputCommitterContainer exten
             for(HCatOutputStorageDriver baseOsd : storageDriversDiscoveredByPath.values()){
                 try {
                     baseOsd.abortOutputCommitterJob(
-                            new TaskAttemptContext(
+                            HCatHadoopShims.Instance.get().createTaskAttemptContext(
                                     jobContext.getConfiguration(), TaskAttemptID.forName(ptnRootLocation)
                             ),state);
                 } catch (Exception e) {
@@ -256,7 +257,7 @@ class FileOutputCommitterContainer exten
                 for(HCatOutputStorageDriver baseOsd : storageDriversDiscoveredByPath.values()){
                     try {
                         baseOsd.cleanupOutputCommitterJob(
-                                new TaskAttemptContext(
+                                HCatHadoopShims.Instance.get().createTaskAttemptContext(
                                         context.getConfiguration(), TaskAttemptID.forName(ptnRootLocation)
                                 ));
                     } catch (Exception e) {

Modified: incubator/hcatalog/trunk/src/java/org/apache/hcatalog/pig/HCatStorer.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/java/org/apache/hcatalog/pig/HCatStorer.java?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/java/org/apache/hcatalog/pig/HCatStorer.java (original)
+++ incubator/hcatalog/trunk/src/java/org/apache/hcatalog/pig/HCatStorer.java Fri Jan 13 20:10:15
2012
@@ -23,17 +23,15 @@ import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hcatalog.common.HCatConstants;
 import org.apache.hcatalog.common.HCatException;
 import org.apache.hcatalog.common.HCatUtil;
 import org.apache.hcatalog.data.schema.HCatSchema;
 import org.apache.hcatalog.mapreduce.HCatOutputFormat;
-import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver;
 import org.apache.hcatalog.mapreduce.OutputJobInfo;
+import org.apache.hcatalog.shims.HCatHadoopShims;
 import org.apache.pig.PigException;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.impl.logicalLayer.FrontendException;
@@ -150,7 +148,8 @@ public class HCatStorer extends HCatBase
       //In local mode, mapreduce will not call OutputCommitter.cleanupJob.
       //Calling it from here so that the partition publish happens.
       //This call needs to be removed after MAPREDUCE-1447 is fixed.
-        getOutputFormat().getOutputCommitter(new TaskAttemptContext(job.getConfiguration(),
new TaskAttemptID())).cleanupJob(job);
+        getOutputFormat().getOutputCommitter(HCatHadoopShims.Instance.get().createTaskAttemptContext(
+        		job.getConfiguration(), new TaskAttemptID())).cleanupJob(job);
       } catch (IOException e) {
         throw new IOException("Failed to cleanup job",e);
       } catch (InterruptedException e) {

Modified: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java
(original)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java
Fri Jan 13 20:10:15 2012
@@ -48,12 +48,14 @@ import org.apache.hcatalog.data.DefaultH
 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;
 
 
 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();
 
   // Generate sample records to compare against
   private byte[][][] getRecords() throws UnsupportedEncodingException {
@@ -99,7 +101,7 @@ public class TestRCFileInputStorageDrive
 
     HCatSchema schema = buildHiveSchema();
     RCFileInputDriver sd = new RCFileInputDriver();
-    JobContext jc = new JobContext(conf, new JobID());
+    JobContext jc = shim.createJobContext(conf, new JobID());
     sd.setInputPath(jc, file.toString());
     InputFormat<?,?> iF = sd.getInputFormat(null);
     InputSplit split = iF.getSplits(jc).get(0);
@@ -107,7 +109,7 @@ public class TestRCFileInputStorageDrive
     sd.setOutputSchema(jc, schema);
     sd.initialize(jc, getProps());
 
-    TaskAttemptContext tac = new TaskAttemptContext(conf, new TaskAttemptID());
+    TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID());
     RecordReader<?,?> rr = iF.createRecordReader(split,tac);
     rr.initialize(split, tac);
     HCatRecord[] tuples = getExpectedRecords();
@@ -125,7 +127,7 @@ public class TestRCFileInputStorageDrive
     BytesRefArrayWritable[] bytesArr = initTestEnvironment();
 
     RCFileInputDriver sd = new RCFileInputDriver();
-    JobContext jc = new JobContext(conf, new JobID());
+    JobContext jc = shim.createJobContext(conf, new JobID());
     sd.setInputPath(jc, file.toString());
     InputFormat<?,?> iF = sd.getInputFormat(null);
     InputSplit split = iF.getSplits(jc).get(0);
@@ -134,7 +136,7 @@ public class TestRCFileInputStorageDrive
 
     sd.initialize(jc, getProps());
     conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR,jc.getConfiguration().get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
-    TaskAttemptContext tac = new TaskAttemptContext(conf, new TaskAttemptID());
+    TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID());
     RecordReader<?,?> rr = iF.createRecordReader(split,tac);
     rr.initialize(split, tac);
     HCatRecord[] tuples = getPrunedRecords();
@@ -154,7 +156,7 @@ public class TestRCFileInputStorageDrive
     BytesRefArrayWritable[] bytesArr = initTestEnvironment();
 
     RCFileInputDriver sd = new RCFileInputDriver();
-    JobContext jc = new JobContext(conf, new JobID());
+    JobContext jc = shim.createJobContext(conf, new JobID());
     sd.setInputPath(jc, file.toString());
     InputFormat<?,?> iF = sd.getInputFormat(null);
     InputSplit split = iF.getSplits(jc).get(0);
@@ -166,7 +168,7 @@ public class TestRCFileInputStorageDrive
     map.put("part1", "first-part");
     sd.setPartitionValues(jc, map);
     conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR,jc.getConfiguration().get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
-    TaskAttemptContext tac = new TaskAttemptContext(conf, new TaskAttemptID());
+    TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID());
     RecordReader<?,?> rr = iF.createRecordReader(split,tac);
     rr.initialize(split, tac);
     HCatRecord[] tuples = getReorderedCols();

Modified: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
(original)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
Fri Jan 13 20:10:15 2012
@@ -45,6 +45,7 @@ import org.apache.hadoop.mapreduce.Recor
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hcatalog.rcfile.RCFileMapReduceInputFormat;
+import org.apache.hcatalog.shims.HCatHadoopShims;
 
   /**
    * TestRCFile.
@@ -232,7 +233,7 @@ import org.apache.hcatalog.rcfile.RCFile
       assertEquals("splits length should be " + splitNumber, splits.size(), splitNumber);
       int readCount = 0;
       for (int i = 0; i < splits.size(); i++) {
-        TaskAttemptContext tac = new TaskAttemptContext(jonconf, new TaskAttemptID());
+        TaskAttemptContext tac = HCatHadoopShims.Instance.get().createTaskAttemptContext(jonconf,
new TaskAttemptID());
         RecordReader<LongWritable, BytesRefArrayWritable> rr = inputFormat.createRecordReader(splits.get(i),
tac);
         rr.initialize(splits.get(i), tac);
         while (rr.nextKeyValue()) {

Modified: incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java
URL: http://svn.apache.org/viewvc/incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java?rev=1231282&r1=1231281&r2=1231282&view=diff
==============================================================================
--- incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java
(original)
+++ incubator/hcatalog/trunk/src/test/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java
Fri Jan 13 20:10:15 2012
@@ -30,7 +30,6 @@ import org.apache.hadoop.hive.serde2.col
 import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hcatalog.common.ErrorType;
 import org.apache.hcatalog.common.HCatConstants;
 import org.apache.hcatalog.common.HCatException;
 import org.apache.hcatalog.common.HCatUtil;
@@ -39,14 +38,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.rcfile.RCFileInputDriver;
-import org.apache.hcatalog.rcfile.RCFileOutputDriver;
+import org.apache.hcatalog.shims.HCatHadoopShims;
 
 public class TestRCFileOutputStorageDriver extends TestCase {
 
   public void testConversion() throws IOException {
     Configuration conf = new Configuration();
-    JobContext jc = new JobContext(conf, new JobID());
+    JobContext jc = HCatHadoopShims.Instance.get().createJobContext(conf, new JobID());
     String jobString = HCatUtil.serialize(OutputJobInfo.create(null,null,null,null,null));
     jc.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO,jobString);
 



Mime
View raw message