You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2013/11/13 17:47:12 UTC

svn commit: r1541609 [1/4] - in /hive/trunk: ./ cli/ hbase-handler/ hbase-handler/src/java/org/apache/hadoop/hive/hbase/ hbase-handler/src/test/org/apache/hadoop/hive/hbase/ hbase-handler/src/test/queries/negative/ hcatalog/ hcatalog/build-support/ant/...

Author: brock
Date: Wed Nov 13 16:47:11 2013
New Revision: 1541609

URL: http://svn.apache.org/r1541609
Log:
HBASE-4388 - Upgrade HBase to 0.96 (Brock Noland, Sushanth Sowmyan, Gunther Hagleitner, and others)

Added:
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataInputInputStream.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataOutputOutputStream.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/PutWritable.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
    hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/gen-java/org/apache/hcatalog/hbase/snapshot/RevisionManagerEndpointProtos.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/RPCConverter.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/apache/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/apache/hcatalog/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/apache/hcatalog/hbase/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/apache/hcatalog/hbase/snapshot/
    hive/trunk/hcatalog/storage-handlers/hbase/src/protobuf/org/apache/hcatalog/hbase/snapshot/RevisionManagerEndpoint.proto
Removed:
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/RevisionManagerProtocol.java
Modified:
    hive/trunk/.gitignore
    hive/trunk/cli/pom.xml
    hive/trunk/hbase-handler/pom.xml
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
    hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
    hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
    hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop.q
    hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q
    hive/trunk/hcatalog/build-support/ant/checkstyle.xml
    hive/trunk/hcatalog/build.properties
    hive/trunk/hcatalog/storage-handlers/hbase/pom.xml
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseBaseOutputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseBulkOutputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseDirectOutputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseHCatStorageHandler.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HBaseInputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/HbaseSnapshotRecordReader.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/ImportSequenceFile.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/RevisionManager.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/RevisionManagerEndpoint.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/RevisionManagerEndpointClient.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/TableSnapshot.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/Transaction.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/java/org/apache/hcatalog/hbase/snapshot/ZKBasedRevisionManager.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/ManyMiniCluster.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseBulkOutputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHBaseDirectOutputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/TestHCatHBaseInputFormat.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestRevisionManager.java
    hive/trunk/hcatalog/storage-handlers/hbase/src/test/org/apache/hcatalog/hbase/snapshot/TestRevisionManagerEndpoint.java
    hive/trunk/itests/hcatalog-unit/pom.xml
    hive/trunk/itests/qtest/pom.xml
    hive/trunk/itests/util/pom.xml
    hive/trunk/pom.xml
    hive/trunk/ql/pom.xml

Modified: hive/trunk/.gitignore
URL: http://svn.apache.org/viewvc/hive/trunk/.gitignore?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/.gitignore (original)
+++ hive/trunk/.gitignore Wed Nov 13 16:47:11 2013
@@ -18,3 +18,10 @@ datanucleus.log
 .arc
 TempStatsStore/
 target/
+ql/TempStatsStore
+hcatalog/hcatalog-pig-adapter/target
+hcatalog/server-extensions/target
+hcatalog/core/target
+hcatalog/webhcat/java-client/target
+hcatalog/storage-handlers/hbase/target
+hcatalog/webhcat/svr/target

Modified: hive/trunk/cli/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/cli/pom.xml?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/cli/pom.xml (original)
+++ hive/trunk/cli/pom.xml Wed Nov 13 16:47:11 2013
@@ -103,12 +103,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase</artifactId>
-      <version>${hbase.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
       <version>${mockito-all.version}</version>
@@ -129,6 +123,12 @@
           <version>${hadoop-20S.version}</version>
          <optional>true</optional>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
    <profile>
@@ -146,6 +146,12 @@
           <version>${hadoop-23.version}</version>
           <optional>true</optional>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>

Modified: hive/trunk/hbase-handler/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/pom.xml?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/pom.xml (original)
+++ hive/trunk/hbase-handler/pom.xml Wed Nov 13 16:47:11 2013
@@ -74,17 +74,6 @@
       <artifactId>commons-logging</artifactId>
       <version>${commons-logging.version}</version>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase</artifactId>
-      <version>${hbase.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
     <!-- test inter-project -->
     <dependency>
       <groupId>junit</groupId>
@@ -92,12 +81,6 @@
       <version>${junit.version}</version>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase</artifactId>
-      <version>${hbase.version}</version>
-      <classifier>tests</classifier>
-    </dependency>
   </dependencies>
 
   <profiles>
@@ -113,6 +96,58 @@
           <version>${hadoop-20S.version}</version>
          <optional>true</optional>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop1-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <classifier>tests</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop1-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-client</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
    <profile>
@@ -130,6 +165,58 @@
           <version>${hadoop-23.version}</version>
           <optional>true</optional>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop2-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <classifier>tests</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop2-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-client</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>

Added: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataInputInputStream.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataInputInputStream.java?rev=1541609&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataInputInputStream.java (added)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataInputInputStream.java Wed Nov 13 16:47:11 2013
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class DataInputInputStream extends InputStream {
+
+  private final DataInput dataInput;
+  public DataInputInputStream(DataInput dataInput) {
+    this.dataInput = dataInput;
+  }
+  @Override
+  public int read() throws IOException {
+    try {
+      return dataInput.readUnsignedByte();
+    } catch (EOFException e) {
+      // contract on EOF differs between DataInput and InputStream
+      return -1;
+    }
+  }
+
+  public static InputStream from(DataInput dataInput) {
+    if(dataInput instanceof InputStream) {
+      return (InputStream)dataInput;
+    }
+    return new DataInputInputStream(dataInput);
+  }
+}

Added: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataOutputOutputStream.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataOutputOutputStream.java?rev=1541609&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataOutputOutputStream.java (added)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/DataOutputOutputStream.java Wed Nov 13 16:47:11 2013
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class DataOutputOutputStream extends OutputStream {
+
+  private final DataOutput dataOutput;
+  public DataOutputOutputStream(DataOutput dataOutput) {
+    this.dataOutput = dataOutput;
+  }
+  @Override
+  public void write(int b) throws IOException {
+    dataOutput.write(b);
+  }
+
+
+  public static OutputStream from(DataOutput dataOutput) {
+    if(dataOutput instanceof OutputStream) {
+      return (OutputStream)dataOutput;
+    }
+    return new DataOutputOutputStream(dataOutput);
+  }
+}

Modified: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java (original)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseSerDe.java Wed Nov 13 16:47:11 2013
@@ -28,7 +28,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -539,11 +538,11 @@ public class HBaseSerDe extends Abstract
   @Override
   public Object deserialize(Writable result) throws SerDeException {
 
-    if (!(result instanceof Result)) {
-      throw new SerDeException(getClass().getName() + ": expects Result!");
+    if (!(result instanceof ResultWritable)) {
+      throw new SerDeException(getClass().getName() + ": expects ResultWritable!");
     }
 
-    cachedHBaseRow.init((Result) result, columnsMapping);
+    cachedHBaseRow.init(((ResultWritable) result).getResult(), columnsMapping);
 
     return cachedHBaseRow;
   }
@@ -555,7 +554,7 @@ public class HBaseSerDe extends Abstract
 
   @Override
   public Class<? extends Writable> getSerializedClass() {
-    return Put.class;
+    return PutWritable.class;
   }
 
   @Override
@@ -605,7 +604,7 @@ public class HBaseSerDe extends Abstract
       throw new SerDeException(e);
     }
 
-    return put;
+    return new PutWritable(put);
   }
 
   private byte [] serializeField(

Modified: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java (original)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java Wed Nov 13 16:47:11 2013
@@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.mapred.TableOutputFormat;
@@ -209,11 +208,7 @@ public class HBaseStorageHandler extends
 
       // ensure the table is online
       new HTable(hbaseConf, tableDesc.getName());
-    } catch (MasterNotRunningException mnre) {
-      throw new MetaException(StringUtils.stringifyException(mnre));
-    } catch (IOException ie) {
-      throw new MetaException(StringUtils.stringifyException(ie));
-    } catch (SerDeException se) {
+    } catch (Exception se) {
       throw new MetaException(StringUtils.stringifyException(se));
     }
   }

Modified: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java (original)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableInputFormat.java Wed Nov 13 16:47:11 2013
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.mapred.Ta
 import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hive.hbase.HBaseSerDe.ColumnMapping;
 import org.apache.hadoop.hive.ql.exec.ExprNodeConstantEvaluator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -77,12 +76,12 @@ import org.apache.hadoop.mapreduce.lib.i
  * such as column pruning and filter pushdown.
  */
 public class HiveHBaseTableInputFormat extends TableInputFormatBase
-    implements InputFormat<ImmutableBytesWritable, Result> {
+    implements InputFormat<ImmutableBytesWritable, ResultWritable> {
 
   static final Log LOG = LogFactory.getLog(HiveHBaseTableInputFormat.class);
 
   @Override
-  public RecordReader<ImmutableBytesWritable, Result> getRecordReader(
+  public RecordReader<ImmutableBytesWritable, ResultWritable> getRecordReader(
     InputSplit split,
     JobConf jobConf,
     final Reporter reporter) throws IOException {
@@ -179,7 +178,7 @@ public class HiveHBaseTableInputFormat e
     final org.apache.hadoop.mapreduce.RecordReader<ImmutableBytesWritable, Result>
     recordReader = createRecordReader(tableSplit, tac);
 
-    return new RecordReader<ImmutableBytesWritable, Result>() {
+    return new RecordReader<ImmutableBytesWritable, ResultWritable>() {
 
       @Override
       public void close() throws IOException {
@@ -192,8 +191,8 @@ public class HiveHBaseTableInputFormat e
       }
 
       @Override
-      public Result createValue() {
-        return new Result();
+      public ResultWritable createValue() {
+        return new ResultWritable(new Result());
       }
 
       @Override
@@ -215,7 +214,7 @@ public class HiveHBaseTableInputFormat e
       }
 
       @Override
-      public boolean next(ImmutableBytesWritable rowKey, Result value) throws IOException {
+      public boolean next(ImmutableBytesWritable rowKey, ResultWritable value) throws IOException {
 
         boolean next = false;
 
@@ -224,7 +223,7 @@ public class HiveHBaseTableInputFormat e
 
           if (next) {
             rowKey.set(recordReader.getCurrentValue().getRow());
-            Writables.copyWritable(recordReader.getCurrentValue(), value);
+            value.setResult(recordReader.getCurrentValue());
           }
         } catch (InterruptedException e) {
           throw new IOException(e);

Modified: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java (original)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHBaseTableOutputFormat.java Wed Nov 13 16:47:11 2013
@@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.mapred.Ta
 import org.apache.hadoop.hbase.mapreduce.TableOutputCommitter;
 import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.hbase.PutWritable;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputFormat;
@@ -48,7 +50,7 @@ import org.apache.hadoop.util.Progressab
  */
 public class HiveHBaseTableOutputFormat extends
     TableOutputFormat<ImmutableBytesWritable> implements
-    OutputFormat<ImmutableBytesWritable, Put> {
+    OutputFormat<ImmutableBytesWritable, Object> {
 
   static final Log LOG = LogFactory.getLog(HiveHBaseTableOutputFormat.class);
   public static final String HBASE_WAL_ENABLED = "hive.hbase.wal.enabled";
@@ -86,7 +88,7 @@ public class HiveHBaseTableOutputFormat 
 
   @Override
   public
-  org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Put>
+  org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Object>
   getRecordWriter(
       FileSystem fileSystem,
       JobConf jobConf,
@@ -99,21 +101,23 @@ public class HiveHBaseTableOutputFormat 
         jobConf, HiveConf.ConfVars.HIVE_HBASE_WAL_ENABLED);
     final HTable table = new HTable(HBaseConfiguration.create(jobConf), hbaseTableName);
     table.setAutoFlush(false);
-    return new MyRecordWriter(table);
+    return new MyRecordWriter(table,walEnabled);
   }
 
   @Override
-   public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException,
+  public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException,
   InterruptedException {
     return new TableOutputCommitter();
-}
+  }
 
 
-  static private class MyRecordWriter implements org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Put> {
+  static private class MyRecordWriter implements org.apache.hadoop.mapred.RecordWriter<ImmutableBytesWritable, Object> {
     private final HTable m_table;
+    private final boolean m_walEnabled;
 
-    public MyRecordWriter(HTable table) {
+    public MyRecordWriter(HTable table, boolean walEnabled) {
       m_table = table;
+      m_walEnabled = walEnabled;
     }
 
     public void close(Reporter reporter)
@@ -122,8 +126,21 @@ public class HiveHBaseTableOutputFormat 
     }
 
     public void write(ImmutableBytesWritable key,
-        Put value) throws IOException {
-      m_table.put(new Put(value));
+        Object value) throws IOException {
+      Put put;
+      if (value instanceof Put){
+        put = (Put)value;
+      } else if (value instanceof PutWritable) {
+        put = new Put(((PutWritable)value).getPut());
+      } else {
+        throw new IllegalArgumentException("Illegal Argument " + (value == null ? "null" : value.getClass().getName()));
+      }
+      if(m_walEnabled) {
+        put.setDurability(Durability.SYNC_WAL);
+      } else {
+        put.setDurability(Durability.SKIP_WAL);
+      }
+      m_table.put(put);
     }
   }
 }

Added: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/PutWritable.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/PutWritable.java?rev=1541609&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/PutWritable.java (added)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/PutWritable.java Wed Nov 13 16:47:11 2013
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
+import org.apache.hadoop.io.Writable;
+
+public class PutWritable implements Writable {
+
+  private Put put;
+
+  public PutWritable() {
+
+  }
+  public PutWritable(Put put) {
+    this.put = put;
+  }
+  public Put getPut() {
+    return put;
+  }
+  @Override
+  public void readFields(final DataInput in)
+  throws IOException {
+    ClientProtos.MutationProto putProto = ClientProtos.MutationProto.parseDelimitedFrom(DataInputInputStream.from(in));
+    int size = in.readInt();
+    if(size < 0) {
+      throw new IOException("Invalid size " + size);
+    }
+    Cell[] kvs = new Cell[size];
+    for (int i = 0; i < kvs.length; i++) {
+      kvs[i] = KeyValue.create(in);
+    }
+    put = ProtobufUtil.toPut(putProto, CellUtil.createCellScanner(kvs));
+  }
+  @Override
+  public void write(final DataOutput out)
+  throws IOException {
+    ProtobufUtil.toMutationNoData(MutationType.PUT, put).writeDelimitedTo(DataOutputOutputStream.from(out));
+    out.writeInt(put.size());
+    CellScanner scanner = put.cellScanner();
+    while(scanner.advance()) {
+      KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.current());
+      KeyValue.write(kv, out);
+    }
+  }
+}

Added: hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java?rev=1541609&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java (added)
+++ hive/trunk/hbase-handler/src/java/org/apache/hadoop/hive/hbase/ResultWritable.java Wed Nov 13 16:47:11 2013
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.io.Writable;
+
+public class ResultWritable implements Writable {
+
+  private Result result;
+
+  public ResultWritable() {
+
+  }
+  public ResultWritable(Result result) {
+    this.result = result;
+  }
+
+  public Result getResult() {
+    return result;
+  }
+  public void setResult(Result result) {
+    this.result = result;
+  }
+  @Override
+  public void readFields(final DataInput in)
+  throws IOException {
+    ClientProtos.Result protosResult = ClientProtos.Result.parseDelimitedFrom(DataInputInputStream.from(in));
+    int size = in.readInt();
+    if(size < 0) {
+      throw new IOException("Invalid size " + size);
+    }
+    Cell[] kvs = new Cell[size];
+    for (int i = 0; i < kvs.length; i++) {
+      kvs[i] = KeyValue.create(in);
+    }
+    result = ProtobufUtil.toResult(protosResult, CellUtil.createCellScanner(kvs));
+  }
+  @Override
+  public void write(final DataOutput out)
+  throws IOException {
+    ProtobufUtil.toResultNoData(result).writeDelimitedTo(DataOutputOutputStream.from(out));
+    out.writeInt(result.size());
+    for(KeyValue kv : result.list()) {
+      KeyValue.write(kv, out);
+    }
+  }
+
+}

Modified: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java (original)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseSerDe.java Wed Nov 13 16:47:11 2013
@@ -217,7 +217,7 @@ public class TestHBaseSerDe extends Test
     assertEquals(9, fieldRefs.size());
 
     // Deserialize
-    Object row = serDe.deserialize(r);
+    Object row = serDe.deserialize(new ResultWritable(r));
     for (int i = 0; i < fieldRefs.size(); i++) {
       Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
       if (fieldData != null) {
@@ -226,9 +226,9 @@ public class TestHBaseSerDe extends Test
       assertEquals("Field " + i, expectedFieldsData[i], fieldData);
     }
     // Serialize
-    assertEquals(Put.class, serDe.getSerializedClass());
-    Put serializedPut = (Put) serDe.serialize(row, oi);
-    assertEquals("Serialized data", p.toString(), serializedPut.toString());
+    assertEquals(PutWritable.class, serDe.getSerializedClass());
+    PutWritable serializedPut = (PutWritable) serDe.serialize(row, oi);
+    assertEquals("Serialized data", p.toString(),String.valueOf(serializedPut.getPut()));
   }
 
   // No specifications default to UTF8 String storage for backwards compatibility
@@ -513,8 +513,8 @@ public class TestHBaseSerDe extends Test
 
     // Deserialize
     for (int i = 0; i < r.length; i++) {
-      Object row = hbaseSerDe.deserialize(r[i]);
-      Put serializedPut = (Put) hbaseSerDe.serialize(row, soi);
+      Object row = hbaseSerDe.deserialize(new ResultWritable(r[i]));
+      Put serializedPut = ((PutWritable) hbaseSerDe.serialize(row, soi)).getPut();
       byte [] rowKey = serializedPut.getRow();
 
       for (int k = 0; k < rowKey.length; k++) {
@@ -669,7 +669,7 @@ public class TestHBaseSerDe extends Test
     assertEquals(9, fieldRefs.size());
 
     // Deserialize
-    Object row = hbaseSerDe.deserialize(r);
+    Object row = hbaseSerDe.deserialize(new ResultWritable(r));
 
     for (int j = 0; j < fieldRefs.size(); j++) {
       Object fieldData = soi.getStructFieldData(row, fieldRefs.get(j));
@@ -687,7 +687,7 @@ public class TestHBaseSerDe extends Test
     }
 
     // Serialize
-    Put serializedPut = (Put) hbaseSerDe.serialize(row, soi);
+    Put serializedPut = ((PutWritable) hbaseSerDe.serialize(row, soi)).getPut();
     assertEquals("Serialized data: ", p.toString(), serializedPut.toString());
   }
 
@@ -775,7 +775,7 @@ public class TestHBaseSerDe extends Test
 
     List<? extends StructField> fieldRefs = soi.getAllStructFieldRefs();
 
-    Object row = serDe.deserialize(r);
+    Object row = serDe.deserialize(new ResultWritable(r));
 
     int j = 0;
 
@@ -803,7 +803,7 @@ public class TestHBaseSerDe extends Test
     SerDeUtils.getJSONString(row, soi);
 
     // Now serialize
-    Put put = (Put) serDe.serialize(row, soi);
+    Put put = ((PutWritable) serDe.serialize(row, soi)).getPut();
 
     if (p != null) {
       assertEquals("Serialized put:", p.toString(), put.toString());

Added: hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java?rev=1541609&view=auto
==============================================================================
--- hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java (added)
+++ hive/trunk/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestPutResultWritable.java Wed Nov 13 16:47:11 2013
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.hbase;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestPutResultWritable {
+
+  @Test
+  public void testResult() throws Exception {
+    // Initialize a result
+    KeyValue[] kvs = new KeyValue[] {
+      new KeyValue(Bytes.toBytes("test-row"), Bytes.toBytes("cfa"),
+        Bytes.toBytes("col1"), Bytes.toBytes("cfacol1")),
+      new KeyValue(Bytes.toBytes("test-row"), Bytes.toBytes("cfa"),
+        Bytes.toBytes("col2"), Bytes.toBytes("cfacol2"))
+    };
+    Result expected = new Result(kvs);
+    ResultWritable actual = copy(new ResultWritable(expected), new ResultWritable());
+    Assert.assertArrayEquals(expected.raw(), actual.getResult().raw());
+
+  }
+
+  @Test
+  public void testPut() throws Exception {
+    byte[] row = Bytes.toBytes("test-row");
+    // Initialize a result
+    KeyValue[] kvs = new KeyValue[] {
+      new KeyValue(row, Bytes.toBytes("cfa"),
+        Bytes.toBytes("col1"), Bytes.toBytes("cfacol1")),
+      new KeyValue(row, Bytes.toBytes("cfa"),
+        Bytes.toBytes("col2"), Bytes.toBytes("cfacol2"))
+    };
+    Put expected = new Put(row);
+    for (int i = 0; i < kvs.length; i++) {
+      expected.add(kvs[i]);
+    }
+    PutWritable actual = copy(new PutWritable(expected), new PutWritable());
+    Assert.assertArrayEquals(expected.getRow(), actual.getPut().getRow());
+    Assert.assertEquals(expected.getFamilyMap(), actual.getPut().getFamilyMap());
+  }
+
+  private <T extends Writable> T copy(T oldWritable, T newWritable) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(baos);
+    oldWritable.write(out);
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream in = new DataInputStream(bais);
+    newWritable.readFields(in);
+    return newWritable;
+  }
+
+}

Modified: hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop.q
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop.q?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop.q (original)
+++ hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop.q Wed Nov 13 16:47:11 2013
@@ -16,11 +16,11 @@ STORED BY 'org.apache.hadoop.hive.hbase.
 WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string")
 TBLPROPERTIES ("hbase.table.name" = "hbase_table_0");
 
-dfs -ls ../build/ql/tmp/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/data/default/hbase_table_0;
 
 DROP DATABASE IF EXISTS hbaseDB CASCADE;
 
-dfs -ls ../build/ql/tmp/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/data/default/hbase_table_0;
 
 
 

Modified: hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q
URL: http://svn.apache.org/viewvc/hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q (original)
+++ hive/trunk/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q Wed Nov 13 16:47:11 2013
@@ -16,11 +16,11 @@ STORED BY 'org.apache.hadoop.hive.hbase.
 WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string")
 TBLPROPERTIES ("hbase.table.name" = "hbase_table_0");
 
-dfs -ls target/tmp/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/data/default/hbase_table_0;
 
 DROP DATABASE IF EXISTS hbaseDB CASCADE;
 
-dfs -ls target/tmp/hbase/hbase/hbase_table_0;
+dfs -ls target/tmp/hbase/data/hbase/default/hbase_table_0;
 
 
 

Modified: hive/trunk/hcatalog/build-support/ant/checkstyle.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/build-support/ant/checkstyle.xml?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hcatalog/build-support/ant/checkstyle.xml (original)
+++ hive/trunk/hcatalog/build-support/ant/checkstyle.xml Wed Nov 13 16:47:11 2013
@@ -62,6 +62,8 @@
           <exclude name="RELEASE_NOTES.txt"/>
           <exclude name="*.patch"/>
           <exclude name="tags"/>
+          <exclude name="**/*.jar"/>
+          <exclude name="**/*.proto"/> <!-- protocol buffer def files -->
           <exclude name="**/*.swp"/> <!-- vim swap files -->
           <exclude name="**/*.orig"/> <!-- files created by patch -->
           <exclude name="**/*.rej"/> <!-- files created by patch -->

Modified: hive/trunk/hcatalog/build.properties
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/build.properties?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hcatalog/build.properties (original)
+++ hive/trunk/hcatalog/build.properties Wed Nov 13 16:47:11 2013
@@ -88,5 +88,4 @@ mvn.deploy.repo.id=apache.snapshots.http
 mvn.deploy.repo.url=https://repository.apache.org/content/repositories/snapshots
 maven-ant-tasks.version=2.1.3
 mvn.local.repo=${user.home}/.m2/repository
-mvn.hadoop.profile=hadoop20
 

Modified: hive/trunk/hcatalog/storage-handlers/hbase/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/storage-handlers/hbase/pom.xml?rev=1541609&r1=1541608&r2=1541609&view=diff
==============================================================================
--- hive/trunk/hcatalog/storage-handlers/hbase/pom.xml (original)
+++ hive/trunk/hcatalog/storage-handlers/hbase/pom.xml Wed Nov 13 16:47:11 2013
@@ -56,17 +56,6 @@
       <version>${guava.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase</artifactId>
-      <version>${hbase.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.apache.thrift</groupId>
       <artifactId>libthrift</artifactId>
       <version>${libthrift.version}</version>
@@ -90,25 +79,18 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase</artifactId>
-      <version>${hbase.version}</version>
-      <classifier>tests</classifier>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>
       <version>${zookeeper.version}</version>
       <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <version>1.3</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -123,6 +105,37 @@
           <artifactId>hadoop-core</artifactId>
           <version>${hadoop-20S.version}</version>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-client</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop1-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <classifier>tests</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop1-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+        </dependency>
         <!-- test -->
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -130,6 +143,27 @@
           <version>${hadoop-20S.version}</version>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop1.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
    <profile>
@@ -137,13 +171,33 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
           <version>${hadoop-23.version}</version>
         </dependency>
         <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-mapreduce-client-core</artifactId>
-          <version>${hadoop-23.version}</version>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-client</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop2-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
         </dependency>
         <!-- test -->
         <dependency>
@@ -160,6 +214,34 @@
           <classifier>tests</classifier>
           <scope>test</scope>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-common</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-server</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-hadoop2-compat</artifactId>
+          <version>${hbase.hadoop2.version}</version>
+          <classifier>tests</classifier>
+          <scope>test</scope>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>