You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2012/11/29 23:19:44 UTC

svn commit: r1415412 [2/2] - in /hbase/trunk: hbase-examples/src/main/java/org/apache/hadoop/hbase/mapreduce/ hbase-server/src/main/java/org/apache/hadoop/hbase/client/ hbase-server/src/main/java/org/apache/hadoop/hbase/io/ hbase-server/src/main/java/o...

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Nov 29 22:19:39 2012
@@ -132,7 +132,6 @@ import org.apache.hadoop.hbase.util.Hash
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.StringUtils;
 import org.cliffc.high_scale_lib.Counter;
 
@@ -2372,7 +2371,7 @@ public class HRegion implements HeapSize
    * @return true if the new put was executed, false otherwise
    */
   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, Writable w,
+      CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
       Integer lockId, boolean writeToWAL)
   throws IOException{
     checkReadOnly();
@@ -2458,8 +2457,7 @@ public class HRegion implements HeapSize
   @SuppressWarnings("unchecked")
   private void doBatchMutate(Mutation mutation, Integer lid) throws IOException,
       DoNotRetryIOException {
-    Pair<Mutation, Integer>[] mutateWithLocks = new Pair[] { new Pair<Mutation, Integer>(mutation,
-        lid) };
+    Pair<Mutation, Integer>[] mutateWithLocks = new Pair[] { new Pair<Mutation, Integer>(mutation, lid) };
     OperationStatus[] batchMutate = this.batchMutate(mutateWithLocks);
     if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
       throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
@@ -2546,13 +2544,13 @@ public class HRegion implements HeapSize
    * @praram now
    * @throws IOException
    */
-  private void put(byte [] family, List<KeyValue> edits, Integer lid)
+  private void put(final byte [] row, byte [] family, List<KeyValue> edits, Integer lid)
   throws IOException {
     Map<byte[], List<KeyValue>> familyMap;
     familyMap = new HashMap<byte[], List<KeyValue>>();
 
     familyMap.put(family, edits);
-    Put p = new Put();
+    Put p = new Put(row);
     p.setFamilyMap(familyMap);
     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
     p.setWriteToWAL(true);
@@ -3912,7 +3910,7 @@ public class HRegion implements HeapSize
       edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
         HConstants.META_VERSION_QUALIFIER, now,
         Bytes.toBytes(HConstants.META_VERSION)));
-      meta.put(HConstants.CATALOG_FAMILY, edits, lid);
+      meta.put(row, HConstants.CATALOG_FAMILY, edits, lid);
     } finally {
       meta.releaseRowLock(lid);
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Nov 29 22:19:39 2012
@@ -3767,7 +3767,7 @@ public class  HRegionServer implements C
     boolean batchContainsPuts = false, batchContainsDelete = false;
     try {
       ActionResult.Builder resultBuilder = ActionResult.newBuilder();
-      NameBytesPair value = ProtobufUtil.toParameter(new Result());
+      NameBytesPair value = ProtobufUtil.toParameter(ClientProtos.Result.newBuilder().build());
       resultBuilder.setValue(value);
       ActionResult result = resultBuilder.build();
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Thu Nov 29 22:19:39 2012
@@ -290,6 +290,7 @@ public class RemoteHTable implements HTa
         // fall through
       case 404:
         return new Result();
+
       case 509:
         try {
           Thread.sleep(sleepTime);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Thu Nov 29 22:19:39 2012
@@ -23,18 +23,12 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
@@ -151,6 +145,8 @@ public class TestSerialization {
       HConstants.EMPTY_END_ROW);
   }
 
+  /*
+   * TODO
   @Test public void testPut() throws Exception{
     byte[] row = "row".getBytes();
     byte[] fam = "fam".getBytes();
@@ -254,6 +250,7 @@ public class TestSerialization {
       }
     }
   }
+  */
 
   @Test public void testGet() throws Exception{
     byte[] row = "row".getBytes();
@@ -347,6 +344,8 @@ public class TestSerialization {
     assertEquals(tr.getMin(), desTr.getMin());
   }
 
+  /*
+   * TODO
   @Test public void testResultEmpty() throws Exception {
     List<KeyValue> keys = new ArrayList<KeyValue>();
     Result r = new Result(keys);
@@ -520,6 +519,7 @@ public class TestSerialization {
     assertTrue(deResults.length == 0);
 
   }
+  */
 
   @Test public void testTimeRange() throws Exception{
     TimeRange tr = new TimeRange(0,5);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java Thu Nov 29 22:19:39 2012
@@ -19,12 +19,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.SmallTests;
@@ -36,31 +30,8 @@ import org.junit.experimental.categories
 @Category(SmallTests.class)
 public class TestAttributes {
   @Test
-  public void testAttributesSerialization() throws IOException {
-    Put put = new Put();
-    put.setAttribute("attribute1", Bytes.toBytes("value1"));
-    put.setAttribute("attribute2", Bytes.toBytes("value2"));
-    put.setAttribute("attribute3", Bytes.toBytes("value3"));
-
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    DataOutput out = new DataOutputStream(byteArrayOutputStream);
-    put.write(out);
-
-    Put put2 = new Put();
-    Assert.assertTrue(put2.getAttributesMap().isEmpty());
-
-    put2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
-
-    Assert.assertNull(put2.getAttribute("absent"));
-    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), put2.getAttribute("attribute1")));
-    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), put2.getAttribute("attribute2")));
-    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value3"), put2.getAttribute("attribute3")));
-    Assert.assertEquals(3, put2.getAttributesMap().size());
-  }
-
-  @Test
   public void testPutAttributes() {
-    Put put = new Put();
+    Put put = new Put(new byte [] {});
     Assert.assertTrue(put.getAttributesMap().isEmpty());
     Assert.assertNull(put.getAttribute("absent"));
 
@@ -108,7 +79,7 @@ public class TestAttributes {
 
   @Test
   public void testDeleteAttributes() {
-    Delete del = new Delete();
+    Delete del = new Delete(new byte [] {});
     Assert.assertTrue(del.getAttributesMap().isEmpty());
     Assert.assertNull(del.getAttribute("absent"));
 
@@ -171,7 +142,7 @@ public class TestAttributes {
 
   @Test
   public void testDeleteId() {
-    Delete delete = new Delete();
+    Delete delete = new Delete(new byte [] {});
     Assert.assertNull("Make sure id is null if unset", delete.toMap().get("id"));
     delete.setId("myId");
     Assert.assertEquals("myId", delete.toMap().get("id"));
@@ -179,7 +150,7 @@ public class TestAttributes {
 
   @Test
   public void testPutId() {
-    Put put = new Put();
+    Put put = new Put(new byte [] {});
     Assert.assertNull("Make sure id is null if unset", put.toMap().get("id"));
     put.setId("myId");
     Assert.assertEquals("myId", put.toMap().get("id"));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java Thu Nov 29 22:19:39 2012
@@ -26,10 +26,14 @@ import java.util.concurrent.ThreadPoolEx
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.ipc.HBaseClient;
+import org.apache.hadoop.hbase.ipc.HBaseServer;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -42,6 +46,10 @@ import static org.junit.Assert.*;
 @Category(MediumTests.class)
 public class TestMultiParallel {
   private static final Log LOG = LogFactory.getLog(TestMultiParallel.class);
+  {
+    ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
+  }
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final byte[] VALUE = Bytes.toBytes("value");
   private static final byte[] QUALIFIER = Bytes.toBytes("qual");
@@ -200,6 +208,12 @@ public class TestMultiParallel {
     table.close();
   }
 
+  @Test (timeout=300000)
+  public void testFlushCommitsNoAbort() throws Exception {
+    LOG.info("test=testFlushCommitsNoAbort");
+    doTestFlushCommits(false);
+  }
+
   /**
    * Only run one Multi test with a forced RegionServer abort. Otherwise, the
    * unit tests will take an unnecessarily long time to run.
@@ -212,12 +226,6 @@ public class TestMultiParallel {
     doTestFlushCommits(true);
   }
 
-  @Test (timeout=300000)
-  public void testFlushCommitsNoAbort() throws Exception {
-    LOG.info("test=testFlushCommitsNoAbort");
-    doTestFlushCommits(false);
-  }
-
   private void doTestFlushCommits(boolean doAbort) throws Exception {
     // Load the data
     LOG.info("get new table");
@@ -249,16 +257,14 @@ public class TestMultiParallel {
     validateLoadedData(table);
 
     // Validate server and region count
-    List<JVMClusterUtil.RegionServerThread> liveRSs =
-      UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
+    List<JVMClusterUtil.RegionServerThread> liveRSs = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
     int count = 0;
     for (JVMClusterUtil.RegionServerThread t: liveRSs) {
       count++;
       LOG.info("Count=" + count + ", Alive=" + t.getRegionServer());
     }
     LOG.info("Count=" + count);
-    Assert.assertEquals("Server count=" + count + ", abort=" + doAbort,
-      (doAbort ? 1 : 2), count);
+    Assert.assertEquals("Server count=" + count + ", abort=" + doAbort, (doAbort? 1 : 2), count);
     for (JVMClusterUtil.RegionServerThread t: liveRSs) {
       int regions = ProtobufUtil.getOnlineRegions(t.getRegionServer()).size();
       Assert.assertTrue("Count of regions=" + regions, regions > 10);
@@ -416,6 +422,7 @@ public class TestMultiParallel {
     validateResult(multiRes[1], QUAL4, Bytes.toBytes("xyz"));
     validateResult(multiRes[0], QUAL2, Bytes.toBytes(2L));
     validateResult(multiRes[0], QUAL3, Bytes.toBytes(1L));
+    table.close();
   }
 
   @Test(timeout=300000)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java?rev=1415412&r1=1415411&r2=1415412&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTimestampsFilter.java Thu Nov 29 22:19:39 2012
@@ -139,7 +139,7 @@ public class TestTimestampsFilter {
     // Request an empty list of versions using the Timestamps filter;
     // Should return none.
     kvs = getNVersions(ht, FAMILY, 2, 2, new ArrayList<Long>());
-    assertEquals(0, kvs.length);
+    assertEquals(0, kvs == null? 0: kvs.length);
 
     //
     // Test the filter using a Scan operation
@@ -272,7 +272,7 @@ public class TestTimestampsFilter {
         // ask for versions that do not exist.
         kvs = getNVersions(ht, cf, rowIdx, colIdx,
                            Arrays.asList(101L, 102L));
-        assertEquals(0, kvs.length);
+        assertEquals(0, kvs == null? 0: kvs.length);
 
         // ask for some versions that exist and some that do not.
         kvs = getNVersions(ht, cf, rowIdx, colIdx,