You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/09/07 07:28:33 UTC

svn commit: r1381881 - in /hbase/trunk: ./ hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/ hbase-hadoop1-compat/ hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/ hbase-hadoop1-compat/src/test/resources/ hbase-hadoop1-compat/src/te...

Author: stack
Date: Fri Sep  7 05:28:31 2012
New Revision: 1381881

URL: http://svn.apache.org/viewvc?rev=1381881&view=rev
Log:
HBASE-6706 Remove TotalOrderPartitioner

Added:
    hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java
    hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java
    hbase/trunk/hbase-hadoop1-compat/src/test/resources/
    hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/
    hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/
    hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims
    hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java
    hbase/trunk/hbase-hadoop2-compat/src/test/resources/
    hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/
    hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/
    hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims
Removed:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/hadoopbackport/
Modified:
    hbase/trunk/hbase-hadoop1-compat/pom.xml
    hbase/trunk/hbase-hadoop2-compat/pom.xml
    hbase/trunk/hbase-server/pom.xml
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
    hbase/trunk/pom.xml

Added: hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java?rev=1381881&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java (added)
+++ hbase/trunk/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/HadoopShims.java Fri Sep  7 05:28:31 2012
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase;
+
+
+/**
+ * A compatibility shim layer for interacting with different versions of Hadoop.
+ */
+//NOTE: we can move this under src/main if main code wants to use this shim layer
+public interface HadoopShims {
+
+  /**
+   * Returns a TaskAttemptContext instance created from the given parameters.
+   * @param job an instance of o.a.h.mapreduce.Job
+   * @param taskId an identifier for the task attempt id. Should be parsable by
+   * TaskAttemptId.forName()
+   * @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
+   */
+  public <T,J> T createTestTaskAttemptContext(final J job, final String taskId);
+
+}

Modified: hbase/trunk/hbase-hadoop1-compat/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/pom.xml?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/pom.xml (original)
+++ hbase/trunk/hbase-hadoop1-compat/pom.xml Fri Sep  7 05:28:31 2012
@@ -59,6 +59,12 @@ limitations under the License.
       <artifactId>hbase-hadoop-compat</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-core</artifactId>
       <version>${hadoop-one.version}</version><!--$NO-MVN-MAN-VER$-->

Added: hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java?rev=1381881&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java Fri Sep  7 05:28:31 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.hadoop.hbase;
+
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+/**
+ * Compatibility shim layer implementation for Hadoop-1.
+ */
+public class HadoopShimsImpl implements HadoopShims {
+
+  /**
+   * Returns a TaskAttemptContext instance created from the given parameters.
+   * @param job an instance of o.a.h.mapreduce.Job
+   * @param taskId an identifier for the task attempt id. Should be parsable by
+   * TaskAttemptId.forName()
+   * @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T, J> T createTestTaskAttemptContext(J job, String taskId) {
+    Job j = (Job)job;
+    return (T)new TaskAttemptContext(j.getConfiguration(), TaskAttemptID.forName(taskId));
+  }
+}

Added: hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims?rev=1381881&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims (added)
+++ hbase/trunk/hbase-hadoop1-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims Fri Sep  7 05:28:31 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.HadoopShimsImpl
\ No newline at end of file

Modified: hbase/trunk/hbase-hadoop2-compat/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/pom.xml?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/pom.xml (original)
+++ hbase/trunk/hbase-hadoop2-compat/pom.xml Fri Sep  7 05:28:31 2012
@@ -113,6 +113,12 @@ limitations under the License.
       <artifactId>hbase-hadoop-compat</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <version>${hadoop-two.version}</version>

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java?rev=1381881&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/HadoopShimsImpl.java Fri Sep  7 05:28:31 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.hadoop.hbase;
+
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+/**
+ * Compatibility shim layer implementation for Hadoop-2.
+ */
+public class HadoopShimsImpl implements HadoopShims {
+
+  /**
+   * Returns a TaskAttemptContext instance created from the given parameters.
+   * @param job an instance of o.a.h.mapreduce.Job
+   * @param taskId an identifier for the task attempt id. Should be parsable by
+   * TaskAttemptId.forName()
+   * @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public <T, J> T createTestTaskAttemptContext(J job, String taskId) {
+    Job j = (Job)job;
+    return (T)new TaskAttemptContextImpl(j.getConfiguration(), TaskAttemptID.forName(taskId));
+  }
+}

Added: hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims?rev=1381881&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/resources/META-INF/services/org.apache.hadoop.hbase.HadoopShims Fri Sep  7 05:28:31 2012
@@ -0,0 +1 @@
+org.apache.hadoop.hbase.HadoopShimsImpl
\ No newline at end of file

Modified: hbase/trunk/hbase-server/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/pom.xml?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/hbase-server/pom.xml (original)
+++ hbase/trunk/hbase-server/pom.xml Fri Sep  7 05:28:31 2012
@@ -268,8 +268,19 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>${compat.module}</artifactId>
-      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>${compat.module}</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
     <!-- General dependencies -->
     <dependency>

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java Fri Sep  7 05:28:31 2012
@@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
@@ -62,11 +61,11 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
 
 /**
  * Writes HFiles. Passed KeyValues must arrive in order.
@@ -83,9 +82,9 @@ public class HFileOutputFormat extends F
   static Log LOG = LogFactory.getLog(HFileOutputFormat.class);
   static final String COMPRESSION_CONF_KEY = "hbase.hfileoutputformat.families.compression";
   TimeRangeTracker trt = new TimeRangeTracker();
-  private static final String DATABLOCK_ENCODING_CONF_KEY = 
+  private static final String DATABLOCK_ENCODING_CONF_KEY =
      "hbase.mapreduce.hfileoutputformat.datablock.encoding";
-  
+
   public RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(final TaskAttemptContext context)
   throws IOException, InterruptedException {
     // Get the path of the temporary output file
@@ -121,7 +120,7 @@ public class HFileOutputFormat extends F
                 + DATABLOCK_ENCODING_CONF_KEY + " : " + dataBlockEncodingStr);
       }
     }
-    
+
     return new RecordWriter<ImmutableBytesWritable, KeyValue>() {
       // Map of families to writers and how much has been output on the writer.
       private final Map<byte [], WriterLength> writers =
@@ -317,13 +316,8 @@ public class HFileOutputFormat extends F
   public static void configureIncrementalLoad(Job job, HTable table)
   throws IOException {
     Configuration conf = job.getConfiguration();
-    Class<? extends Partitioner> topClass;
-    try {
-      topClass = getTotalOrderPartitionerClass();
-    } catch (ClassNotFoundException e) {
-      throw new IOException("Failed getting TotalOrderPartitioner", e);
-    }
-    job.setPartitionerClass(topClass);
+
+    job.setPartitionerClass(TotalOrderPartitioner.class);
     job.setOutputKeyClass(ImmutableBytesWritable.class);
     job.setOutputValueClass(KeyValue.class);
     job.setOutputFormatClass(HFileOutputFormat.class);
@@ -355,11 +349,7 @@ public class HFileOutputFormat extends F
 
     URI cacheUri;
     try {
-      // Below we make explicit reference to the bundled TOP.  Its cheating.
-      // We are assume the define in the hbase bundled TOP is as it is in
-      // hadoop (whether 0.20 or 0.22, etc.)
-      cacheUri = new URI(partitionsPath.toString() + "#" +
-        org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner.DEFAULT_PATH);
+      cacheUri = new URI(partitionsPath.toString() + "#" + TotalOrderPartitioner.DEFAULT_PATH);
     } catch (URISyntaxException e) {
       throw new IOException(e);
     }
@@ -374,27 +364,6 @@ public class HFileOutputFormat extends F
   }
 
   /**
-   * If > hadoop 0.20, then we want to use the hadoop TotalOrderPartitioner.
-   * If 0.20, then we want to use the TOP that we have under hadoopbackport.
-   * This method is about hbase being able to run on different versions of
-   * hadoop.  In 0.20.x hadoops, we have to use the TOP that is bundled with
-   * hbase.  Otherwise, we use the one in Hadoop.
-   * @return Instance of the TotalOrderPartitioner class
-   * @throws ClassNotFoundException If can't find a TotalOrderPartitioner.
-   */
-  private static Class<? extends Partitioner> getTotalOrderPartitionerClass()
-  throws ClassNotFoundException {
-    Class<? extends Partitioner> clazz = null;
-    try {
-      clazz = (Class<? extends Partitioner>) Class.forName("org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner");
-    } catch (ClassNotFoundException e) {
-      clazz =
-        (Class<? extends Partitioner>) Class.forName("org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner");
-    }
-    return clazz;
-  }
-
-  /**
    * Run inside the task to deserialize column family to compression algorithm
    * map from the
    * configuration.

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java Fri Sep  7 05:28:31 2012
@@ -26,7 +26,6 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.lang.reflect.Constructor;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -43,11 +42,13 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HadoopShims;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.PerformanceEvaluation;
@@ -74,7 +75,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -180,7 +180,7 @@ public class TestHFileOutputFormat  {
     try {
       Job job = new Job(conf);
       FileOutputFormat.setOutputPath(job, dir);
-      context = getTestTaskAttemptContext(job);
+      context = createTestTaskAttemptContext(job);
       HFileOutputFormat hof = new HFileOutputFormat();
       writer = hof.getRecordWriter(context);
       final byte [] b = Bytes.toBytes("b");
@@ -208,29 +208,10 @@ public class TestHFileOutputFormat  {
     }
   }
 
-  /**
-   * @return True if the available mapreduce is post-0.20.
-   */
-  private static boolean isPost020MapReduce() {
-    // Here is a coarse test for post 0.20 hadoop; TAC became an interface.
-    return TaskAttemptContext.class.isInterface();
-  }
-
-  private TaskAttemptContext getTestTaskAttemptContext(final Job job)
+  private TaskAttemptContext createTestTaskAttemptContext(final Job job)
   throws IOException, Exception {
-    TaskAttemptContext context;
-    if (isPost020MapReduce()) {
-      TaskAttemptID id =
-        TaskAttemptID.forName("attempt_200707121733_0001_m_000000_0");
-      Class<?> clazz =
-        Class.forName("org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl");
-      Constructor<?> c = clazz.
-          getConstructor(Configuration.class, TaskAttemptID.class);
-      context = (TaskAttemptContext)c.newInstance(job.getConfiguration(), id);
-    } else {
-      context = org.apache.hadoop.hbase.mapreduce.hadoopbackport.InputSampler.
-        getTaskAttemptContext(job);
-    }
+    HadoopShims hadoop = CompatibilitySingletonFactory.getInstance(HadoopShims.class);
+    TaskAttemptContext context = hadoop.createTestTaskAttemptContext(job, "attempt_200707121733_0001_m_000000_0");
     return context;
   }
 
@@ -250,7 +231,7 @@ public class TestHFileOutputFormat  {
       // build a record writer using HFileOutputFormat
       Job job = new Job(conf);
       FileOutputFormat.setOutputPath(job, dir);
-      context = getTestTaskAttemptContext(job);
+      context = createTestTaskAttemptContext(job);
       HFileOutputFormat hof = new HFileOutputFormat();
       writer = hof.getRecordWriter(context);
 
@@ -593,7 +574,7 @@ public class TestHFileOutputFormat  {
       setupRandomGeneratorMapper(job);
       HFileOutputFormat.configureIncrementalLoad(job, table);
       FileOutputFormat.setOutputPath(job, dir);
-      context = getTestTaskAttemptContext(job);
+      context = createTestTaskAttemptContext(job);
       HFileOutputFormat hof = new HFileOutputFormat();
       writer = hof.getRecordWriter(context);
 

Modified: hbase/trunk/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/pom.xml?rev=1381881&r1=1381880&r2=1381881&view=diff
==============================================================================
--- hbase/trunk/pom.xml (original)
+++ hbase/trunk/pom.xml Fri Sep  7 05:28:31 2012
@@ -881,10 +881,24 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-hadoop-compat</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>${compat.module}</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>${compat.module}</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
         <artifactId>hbase-server</artifactId>
         <groupId>org.apache.hbase</groupId>
         <version>${project.version}</version>