You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/03/12 22:17:20 UTC

svn commit: r1576909 [15/18] - in /hbase/branches/0.89-fb/src: ./ examples/thrift/ main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/avro/ main/java/org/apache/hadoop/hbase/avro/generated/ main/java/org/apache/hadoop/hbase/client/ ma...

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java Wed Mar 12 21:17:13 2014
@@ -88,16 +88,16 @@ public class TestAdmin {
     tables = this.admin.listTables();
     assertEquals(numTables + 1, tables.length);
   }
-
+  
   @Test
   public void testGetTableRegions() throws IOException {
     TEST_UTIL.createTable(Bytes.toBytes("testCreateNormalTable"),
       HConstants.CATALOG_FAMILY);
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
-    List<Pair<HRegionInfo,HServerAddress>>  tableRegions =
+    List<Pair<HRegionInfo,HServerAddress>>  tableRegions = 
         master.getTableRegions(Bytes.toBytes("testCreateTable"));
     assertTrue(tableRegions.size() != 0);
-    List<Pair<HRegionInfo,HServerAddress>>  metaRegions =
+    List<Pair<HRegionInfo,HServerAddress>>  metaRegions = 
         master.getTableRegions(HConstants.META_TABLE_NAME);
     assertTrue(metaRegions.size() != 0);
   }
@@ -288,11 +288,11 @@ public class TestAdmin {
       ok = true;
     }
     
-    // with online schema change it is possible to add column
+    // with online schema change it is possible to add column 
     // without disabling the table
     assertEquals(true, ok);
     this.admin.enableTable(table);
-    ok = true;
+    ok = true; 
     //Test that table is enabled
     try {
       ht.get(get);
@@ -323,7 +323,7 @@ public class TestAdmin {
     splitTest(null);
     splitTest(Bytes.toBytes("pwn"));
   }
-
+  
   void splitTest(byte[] splitPoint) throws Exception {
     byte [] familyName = HConstants.CATALOG_FAMILY;
     byte [] tableName = Bytes.toBytes("testForceSplit");
@@ -344,12 +344,12 @@ public class TestAdmin {
           }
         }
       }
-
+  
       // get the initial layout (should just be one region)
       Map<HRegionInfo,HServerAddress> m = table.getRegionsInfo();
       System.out.println("Initial regions (" + m.size() + "): " + m);
       assertTrue(m.size() == 1);
-
+  
       // Verify row count
       Scan scan = new Scan();
       ResultScanner scanner = table.getScanner(scan);
@@ -359,13 +359,13 @@ public class TestAdmin {
       }
       scanner.close();
       assertEquals(rowCount, rows);
-
+  
       // Have an outstanding scan going on to make sure we can scan over splits.
       scan = new Scan();
       scanner = table.getScanner(scan);
       // Scan first row so we are into first region before split happens.
       scanner.next();
-
+  
       final AtomicInteger count = new AtomicInteger(0);
       Thread t = new Thread("CheckForSplit") {
         public void run() {
@@ -397,7 +397,7 @@ public class TestAdmin {
         admin.split(tableName);
       }
       t.join();
-
+  
       // Verify row count
       rows = 1; // We counted one row above.
       for (@SuppressWarnings("unused") Result result : scanner) {
@@ -409,7 +409,7 @@ public class TestAdmin {
       }
       scanner.close();
       assertEquals(rowCount, rows);
-
+      
       if (splitPoint != null) {
         // make sure the split point matches our explicit configuration
         Map<HRegionInfo, HServerAddress> regions = null;
@@ -420,9 +420,9 @@ public class TestAdmin {
         }
         assertEquals(2, regions.size());
         HRegionInfo[] r = regions.keySet().toArray(new HRegionInfo[0]);
-        assertEquals(Bytes.toString(splitPoint),
+        assertEquals(Bytes.toString(splitPoint), 
             Bytes.toString(r[0].getEndKey()));
-        assertEquals(Bytes.toString(splitPoint),
+        assertEquals(Bytes.toString(splitPoint), 
             Bytes.toString(r[1].getStartKey()));
         LOG.debug("Properly split on " + Bytes.toString(splitPoint));
       }
@@ -601,7 +601,7 @@ public class TestAdmin {
         "testTableNotFoundExceptionWithoutAnyTables");
   }
 
-  @Test
+  @Test(timeout = 300000)
   public void testHundredsOfTable() throws IOException{
     final int times = 100;
     HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
@@ -640,7 +640,7 @@ public class TestAdmin {
 
     assertEquals(htd.compareTo(confirmedHtd), 0);
   }
-
+  
   @Test
   public void testOnlineChangeTableSchema() throws IOException,
       InterruptedException {
@@ -679,7 +679,7 @@ public class TestAdmin {
     List<Pair<HRegionInfo,HServerAddress>> regionToRegionServer = master.getTableRegions(tableName);
     // check if all regions have the column the correct schema.
     for (Pair<HRegionInfo, HServerAddress> p : regionToRegionServer) {
-      HRegionInfo regionInfo = p.getFirst();
+      HRegionInfo regionInfo = p.getFirst();  
       HTableDescriptor modifiedHtd = regionInfo.getTableDesc();
       // ensure that the Htable descriptor on the master and the region servers
       // of all regions is the same

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java Wed Mar 12 21:17:13 2014
@@ -19,9 +19,10 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Arrays;
-
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -39,8 +40,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.junit.Assert.assertTrue;
-
 /**
  * Run tests that use the HBase clients; {@link HTable} and {@link HTablePool}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
@@ -68,7 +67,7 @@ public class TestBatchedUpload {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
+  @Test(timeout=100000)
   public void testBatchedUpload() throws Exception {
     byte [] TABLE = Bytes.toBytes("testBatchedUpload");
     int NUM_REGIONS = 10;
@@ -88,7 +87,7 @@ public class TestBatchedUpload {
     ht.close();
   }
 
-  @Test
+  @Test(timeout=100000)
   /*
    * Test to make sure that if a region moves benignly, and both
    * the source and dest region servers are alive, then the batch
@@ -203,10 +202,10 @@ public class TestBatchedUpload {
     for(long i = 0; i < numRows; i++) {
       byte [] rowKey = longToByteArrayKey(i);
 
-      Get get = new Get(rowKey);
+      Get.Builder get = new Get.Builder(rowKey);
       get.addColumn(FAMILY, QUALIFIER);
       get.setMaxVersions(1);
-      Result result = table.get(get);
+      Result result = table.get(get.create());
       
       assertTrue(Arrays.equals(rowKey, result.getValue(FAMILY, QUALIFIER)));
     }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestClientLocalScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestClientLocalScanner.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestClientLocalScanner.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestClientLocalScanner.java Wed Mar 12 21:17:13 2014
@@ -76,7 +76,7 @@ public class TestClientLocalScanner {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
+  @Test(timeout=200000)
   public void testCompareLocalScanToRemoteScan() throws IOException {
     byte [] name = Bytes.toBytes("testCompareLocalScanToRemoteScan");
     HTable t = TEST_UTIL.createTable(name, new byte[][] {FAMILY, FAMILY2});

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestDelete.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestDelete.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestDelete.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestDelete.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2013 The Apache Software Foundation
+ *
+ * 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.client;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Test;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+
+public class TestDelete extends TestCase {
+
+  /**
+   * Test to verify if the serialization and deserialization of Delete objects
+   * works fine when using Swift.
+   * @throws Exception
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    ThriftCodec<Delete> codec = new ThriftCodecManager().getCodec(Delete.class);
+    TMemoryBuffer transport = new TMemoryBuffer(100*1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+
+    byte[] row = Bytes.toBytes("row");
+    Map<byte[], List<KeyValue>> familyMap =
+      new TreeMap<byte[], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+    familyMap.put(row, createDummyKVs(row));
+    long lockId = 5;
+    boolean writeToWAL = true;
+    Delete del = new Delete(row, System.currentTimeMillis(),
+                            familyMap, lockId, writeToWAL);
+    codec.write(del, protocol);
+    Delete delCopy = codec.read(protocol);
+    Assert.assertEquals(del, delCopy);
+  }
+
+  @Test
+  public void testIsDummy() throws Exception {
+    Assert.assertTrue("Delete.isDummy", new Delete().isDummy());
+    Assert.assertTrue("Delete.isDummy", new Delete(new byte[0]).isDummy());
+  }
+
+  /**
+   * Test if the Builder is functionally equivalent to the constructor.
+   * @throws Exception
+   */
+  @Test
+  public void testBuilder() throws Exception {
+    byte[] row = Bytes.toBytes("row");
+    Map<byte[], List<KeyValue>> familyMap =
+      new TreeMap<byte[], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+    familyMap.put(row, createDummyKVs(row));
+    long currentMs = System.currentTimeMillis();
+    long lockId = 5;
+    boolean writeToWAL = true;
+    // Create a Delete the standard way
+    Delete del = new Delete(row, currentMs, familyMap, lockId, writeToWAL);
+
+    // Now use a builder to create one, with the same parameters.
+    Delete del2 = new Delete.Builder()
+                        .setRow(row)
+                        .setFamilyMap(familyMap)
+                        .setTimeStamp(currentMs)
+                        .setLockId(lockId)
+                        .setWriteToWAL(writeToWAL)
+                        .create();
+
+    // Check if they match.
+    Assert.assertEquals(del, del2);
+  }
+
+  public static List<KeyValue> createDummyKVs(byte[] row) {
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    for (int i = 0; i < 10; i++) {
+      byte[] family = Bytes.toBytes("fam" + i);
+      byte[] qualifier = Bytes.toBytes("q" + i);
+      byte[] value = Bytes.toBytes("v" + i);
+      kvs.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(),
+                            KeyValue.Type.Delete, value));
+    }
+    return kvs;
+  }
+
+  /**
+   * Utility method that can create n dummy deletes
+   *
+   * @param n
+   * @return List of Delete
+   */
+  public static List<Delete> createDummyDeletes(int n) {
+    List<Delete> deletes = new ArrayList<>();
+    for (int i = 0; i < n; i++) {
+      byte[] row = Bytes.toBytes("row" + i);
+      Map<byte[], List<KeyValue>> familyMap =
+          new TreeMap<byte[], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+      familyMap.put(row, createDummyKVs(row));
+      Delete del = new Delete.Builder().setRow(row).setFamilyMap(familyMap)
+          .setTimeStamp(System.currentTimeMillis()).setLockId(1)
+          .setWriteToWAL(false).create();
+      deletes.add(del);
+    }
+    return deletes;
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java Wed Mar 12 21:17:13 2014
@@ -167,12 +167,12 @@ public class TestFastFail {
     LOG.info("Proceeding");
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailNormalWithoutClearCache() throws Exception {
     testFastFailNormal(false);
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailNormalWithClearCache() throws Exception {
     testFastFailNormal(true);
   }
@@ -197,12 +197,12 @@ public class TestFastFail {
     } while (killedCnt < 1);
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailMetaWithoutClearCache() throws Exception {
     testFastFailMeta(false);
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailMetaWithClearCache() throws Exception {
     testFastFailMeta(true);
   }
@@ -223,12 +223,12 @@ public class TestFastFail {
     } while (true);
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailRootWithoutClearMeta() throws Exception {
     testFastFailRoot(false);
   }
 
-  @Test(timeout = 100000)
+  @Test(timeout = 180000)
   public void testFastFailRootWithClearMeta() throws Exception {
     testFastFailRoot(true);
   }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Wed Mar 12 21:17:13 2014
@@ -143,13 +143,13 @@ public class TestFromClientSide {
     put.add(FAMILY, null, value);
     ht.put(put);
    
-    Get g1 = new Get(r1);
+    Get.Builder g1 = new Get.Builder(r1);
     g1.addColumn(FAMILY, null);
-    result1 = ht.get(g1);
+    result1 = ht.get(g1.create());
     
-    Get g2 = new Get(r1);
+    Get.Builder g2 = new Get.Builder(r1);
     g2.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
-    result2 = ht.get(g2);
+    result2 = ht.get(g2.create());
     
     assertEquals(result1.getBytes(), result2.getBytes());
     assertEquals(1, result2.raw().length);
@@ -160,13 +160,13 @@ public class TestFromClientSide {
     ht.put(put);
     ht.flushCommits();
     
-    g1 = new Get(r2);
+    g1 = new Get.Builder(r2);
     g1.addColumn(FAMILY, null);
-    result1 = ht.get(g1);
+    result1 = ht.get(g1.create());
     
-    g2 = new Get(r2);
+    g2 = new Get.Builder(r2);
     g2.addColumn(FAMILY, HConstants.EMPTY_BYTE_ARRAY);
-    result2 = ht.get(g2);
+    result2 = ht.get(g2.create());
     
     assertEquals(result1.getBytes(), result2.getBytes());
     assertEquals(1, result2.raw().length);
@@ -348,7 +348,7 @@ public class TestFromClientSide {
     int rowCount = TEST_UTIL.loadTable(t, FAMILY);
     assertRowCount(t, rowCount);
     // Split the table.  Should split on a reasonable key; 'lqj'
-    Map<HRegionInfo, HServerAddress> regions  = splitTable(t);
+    Map<HRegionInfo, HServerAddress> regions = splitTable(t);
     assertRowCount(t, rowCount);
     // Get end key of first region.
     byte [] endKey = regions.keySet().iterator().next().getEndKey();
@@ -399,6 +399,7 @@ public class TestFromClientSide {
    * @return Count of rows loaded.
    * @throws IOException
    */
+  @SuppressWarnings("unused")
   private int loadTable(final HTable t) throws IOException {
     // Add data to table.
     byte[] k = new byte[3];
@@ -440,6 +441,7 @@ public class TestFromClientSide {
       Bytes.BYTES_COMPARATOR.compare(key, new byte [] {'a', 'a', 'a'}) >= 0);
     LOG.info("Key=" + Bytes.toString(key));
     Scan s = startRow == null? new Scan(): new Scan(startRow);
+    s.setCaching(1000);
     Filter f = new RowFilter(op, new BinaryComparator(key));
     f = new WhileMatchFilter(f);
     s.setFilter(f);
@@ -467,7 +469,9 @@ public class TestFromClientSide {
 
   private void assertRowCount(final HTable t, final int expected)
   throws IOException {
-    assertEquals(expected, countRows(t, new Scan()));
+    Scan s = new Scan();
+    s.setCaching(1000);
+    assertEquals(expected, countRows(t, s));
   }
 
   /*
@@ -625,18 +629,18 @@ public class TestFromClientSide {
     byte [][] ROWS = makeN(ROW, 4);
 
     // Try to get a row on an empty table
-    Get get = new Get(ROWS[0]);
-    Result result = ht.get(get);
+    Get.Builder get = new Get.Builder(ROWS[0]);
+    Result result = ht.get(get.create());
     assertEmptyResult(result);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addFamily(FAMILY);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILY, QUALIFIER);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     Scan scan = new Scan();
@@ -670,18 +674,18 @@ public class TestFromClientSide {
 
     // Try to get empty rows around it
 
-    get = new Get(ROWS[1]);
-    result = ht.get(get);
+    get = new Get.Builder(ROWS[1]);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addFamily(FAMILY);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
-    get = new Get(ROWS[3]);
+    get = new Get.Builder(ROWS[3]);
     get.addColumn(FAMILY, QUALIFIER);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to scan empty rows around it
@@ -696,18 +700,18 @@ public class TestFromClientSide {
 
     // Make sure we can actually get the row
 
-    get = new Get(ROWS[2]);
-    result = ht.get(get);
+    get = new Get.Builder(ROWS[2]);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
 
-    get = new Get(ROWS[2]);
+    get = new Get.Builder(ROWS[2]);
     get.addFamily(FAMILY);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
 
-    get = new Get(ROWS[2]);
+    get = new Get.Builder(ROWS[2]);
     get.addColumn(FAMILY, QUALIFIER);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[2], FAMILY, QUALIFIER, VALUE);
 
     // Make sure we can scan the row
@@ -809,7 +813,7 @@ public class TestFromClientSide {
 
     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
 
-    Get get;
+    Get.Builder get;
     Scan scan;
     Delete delete;
     Put put;
@@ -894,9 +898,9 @@ public class TestFromClientSide {
     ht.delete(delete);
 
     // Try to get deleted column
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[7]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to scan deleted column
@@ -906,14 +910,14 @@ public class TestFromClientSide {
     assertNullResult(result);
 
     // Make sure we can still get a column before it and after it
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
 
     // Make sure we can still scan a column before it and after it
@@ -935,9 +939,9 @@ public class TestFromClientSide {
     ht.delete(delete);
 
     // Try to get deleted column
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[8]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to scan deleted column
@@ -947,14 +951,14 @@ public class TestFromClientSide {
     assertNullResult(result);
 
     // Make sure we can still get a column before it and after it
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[6]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[6], VALUES[6]);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
 
     // Make sure we can still scan a column before it and after it
@@ -977,21 +981,21 @@ public class TestFromClientSide {
     ht.delete(delete);
 
     // Try to get storefile column in deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to get memstore column in deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to get deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addFamily(FAMILIES[4]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to scan storefile column in deleted family
@@ -1013,14 +1017,14 @@ public class TestFromClientSide {
     assertNullResult(result);
 
     // Make sure we can still get another family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
 
     // Make sure we can still scan another family
@@ -1041,21 +1045,21 @@ public class TestFromClientSide {
     TEST_UTIL.flush();
 
     // Try to get storefile column in deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[4]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to get memstore column in deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[4], QUALIFIERS[3]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to get deleted family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addFamily(FAMILIES[4]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEmptyResult(result);
 
     // Try to scan storefile column in deleted family
@@ -1077,14 +1081,14 @@ public class TestFromClientSide {
     assertNullResult(result);
 
     // Make sure we can still get another family
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[2], QUALIFIERS[2]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[2], QUALIFIERS[2], VALUES[2]);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addColumn(FAMILIES[6], QUALIFIERS[9]);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
 
     // Make sure we can still scan another family
@@ -1140,8 +1144,8 @@ public class TestFromClientSide {
       delete.deleteColumns(FAMILY, null);
       ht.delete(delete);
 
-      Get get = new Get(ROW);
-      Result result = ht.get(get);
+      Get.Builder get = new Get.Builder(ROW);
+      Result result = ht.get(get.create());
       assertEmptyResult(result);
     }
 
@@ -1185,9 +1189,9 @@ public class TestFromClientSide {
       put.add(FAMILY, QUALIFIER, null);
       ht.put(put);
 
-      Get get = new Get(ROW);
+      Get.Builder get = new Get.Builder(ROW);
       get.addColumn(FAMILY, QUALIFIER);
-      Result result = ht.get(get);
+      Result result = ht.get(get.create());
       assertSingleResult(result, ROW, FAMILY, QUALIFIER, null);
 
       Scan scan = new Scan();
@@ -1199,8 +1203,8 @@ public class TestFromClientSide {
       delete.deleteColumns(FAMILY, QUALIFIER);
       ht.delete(delete);
 
-      get = new Get(ROW);
-      result = ht.get(get);
+      get = new Get.Builder(ROW);
+      result = ht.get(get.create());
       assertEmptyResult(result);
 
     } catch(Exception e) {
@@ -1244,10 +1248,10 @@ public class TestFromClientSide {
     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
 
     // Ensure maxVersions in query is respected
-    Get get = new Get(ROW);
+    Get.Builder get = new Get.Builder(ROW);
     get.addColumn(FAMILY, QUALIFIER);
     get.setMaxVersions(2);
-    Result result = ht.get(get);
+    Result result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[4], STAMPS[5]},
         new byte[][] {VALUES[4], VALUES[5]},
@@ -1285,10 +1289,10 @@ public class TestFromClientSide {
     scanVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[6]);
 
     // Ensure maxVersions in query is respected
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILY, QUALIFIER);
     get.setMaxVersions(2);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[4], STAMPS[5]},
         new byte[][] {VALUES[4], VALUES[5]},
@@ -1315,10 +1319,10 @@ public class TestFromClientSide {
     ht.put(put);
 
     // Ensure maxVersions in query is respected
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILY, QUALIFIER);
     get.setMaxVersions();
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
@@ -1333,9 +1337,9 @@ public class TestFromClientSide {
         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
         0, 7);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.setMaxVersions();
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8]},
@@ -1377,10 +1381,10 @@ public class TestFromClientSide {
     put.add(FAMILY, QUALIFIER, STAMPS[15], VALUES[15]);
     ht.put(put);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILY, QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[7], STAMPS[8], STAMPS[9], STAMPS[11], STAMPS[13], STAMPS[15]},
         new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
@@ -1402,10 +1406,10 @@ public class TestFromClientSide {
     ht.delete(delete);
 
     // Test that it's gone
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILY, QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILY, QUALIFIER,
         new long [] {STAMPS[1], STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6], STAMPS[8], STAMPS[9], STAMPS[13], STAMPS[15]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[8], VALUES[9], VALUES[13], VALUES[15]},
@@ -1452,19 +1456,19 @@ public class TestFromClientSide {
 
     // Family0
 
-    Get get = new Get(ROW);
+    Get.Builder get = new Get.Builder(ROW);
     get.addColumn(FAMILIES[0], QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    Result result = ht.get(get);
+    Result result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
         new long [] {STAMPS[1]},
         new byte[][] {VALUES[1]},
         0, 0);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[0]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
         new long [] {STAMPS[1]},
         new byte[][] {VALUES[1]},
@@ -1490,19 +1494,19 @@ public class TestFromClientSide {
 
     // Family1
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILIES[1], QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
         0, 2);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[1]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[1], QUALIFIER,
         new long [] {STAMPS[1], STAMPS[2], STAMPS[3]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
@@ -1528,19 +1532,19 @@ public class TestFromClientSide {
 
     // Family2
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILIES[2], QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
         0, 4);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[2], QUALIFIER,
         new long [] {STAMPS[2], STAMPS[3], STAMPS[4], STAMPS[5], STAMPS[6]},
         new byte[][] {VALUES[2], VALUES[3], VALUES[4], VALUES[5], VALUES[6]},
@@ -1566,27 +1570,27 @@ public class TestFromClientSide {
 
     // Try all families
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 9 keys but received " + result.size(),
         result.size() == 9);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[0]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 9 keys but received " + result.size(),
         result.size() == 9);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILIES[0], QUALIFIER);
     get.addColumn(FAMILIES[1], QUALIFIER);
     get.addColumn(FAMILIES[2], QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 9 keys but received " + result.size(),
         result.size() == 9);
 
@@ -1636,10 +1640,10 @@ public class TestFromClientSide {
     delete.deleteFamily(FAMILIES[0], ts[0]);
     ht.delete(delete);
 
-    Get get = new Get(ROW);
+    Get.Builder get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[0]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    Result result = ht.get(get);
+    Result result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
         new long [] {ts[1]},
         new byte[][] {VALUES[1]},
@@ -1668,10 +1672,10 @@ public class TestFromClientSide {
     delete.deleteColumn(FAMILIES[0], QUALIFIER); // ts[4]
     ht.delete(delete);
 
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addColumn(FAMILIES[0], QUALIFIER);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
         new long [] {ts[1], ts[2], ts[3]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
@@ -1708,10 +1712,10 @@ public class TestFromClientSide {
     // It used to be due to the internal implementation of Get, that
     // the Get() call would return ts[4] UNLIKE the Scan below. With
     // the switch to using Scan for Get this is no longer the case.
-    get = new Get(ROW);
+    get = new Get.Builder(ROW);
     get.addFamily(FAMILIES[0]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertNResult(result, ROW, FAMILIES[0], QUALIFIER,
         new long [] {ts[1], ts[2], ts[3]},
         new byte[][] {VALUES[1], VALUES[2], VALUES[3]},
@@ -1752,11 +1756,11 @@ public class TestFromClientSide {
     ht.put(put);
 
     // Assert that above went in.
-    get = new Get(ROWS[2]);
+    get = new Get.Builder(ROWS[2]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 4 key but received " + result.size() + ": " + result,
         result.size() == 4);
 
@@ -1774,11 +1778,11 @@ public class TestFromClientSide {
     delete.deleteColumn(FAMILIES[2], QUALIFIER);
     ht.delete(delete);
 
-    get = new Get(ROWS[0]);
+    get = new Get.Builder(ROWS[0]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 2 keys but received " + result.size(),
         result.size() == 2);
     assertNResult(result, ROWS[0], FAMILIES[1], QUALIFIER,
@@ -1798,11 +1802,11 @@ public class TestFromClientSide {
         new byte[][] {VALUES[0], VALUES[1]},
         0, 1);
 
-    get = new Get(ROWS[1]);
+    get = new Get.Builder(ROWS[1]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 2 keys but received " + result.size(),
         result.size() == 2);
 
@@ -1814,11 +1818,11 @@ public class TestFromClientSide {
     assertTrue("Expected 2 keys but received " + result.size(),
         result.size() == 2);
 
-    get = new Get(ROWS[2]);
+    get = new Get.Builder(ROWS[2]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertEquals(1, result.size());
     assertNResult(result, ROWS[2], FAMILIES[2], QUALIFIER,
         new long [] {ts[2]},
@@ -1851,19 +1855,19 @@ public class TestFromClientSide {
     put.add(FAMILIES[2], QUALIFIER, VALUES[2]);
     ht.put(put);
 
-    get = new Get(ROWS[3]);
+    get = new Get.Builder(ROWS[3]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 1 key but received " + result.size(),
         result.size() == 1);
 
-    get = new Get(ROWS[4]);
+    get = new Get.Builder(ROWS[4]);
     get.addFamily(FAMILIES[1]);
     get.addFamily(FAMILIES[2]);
     get.setMaxVersions(Integer.MAX_VALUE);
-    result = ht.get(get);
+    result = ht.get(get.create());
     assertTrue("Expected 2 keys but received " + result.size(),
         result.size() == 2);
 
@@ -1906,9 +1910,9 @@ public class TestFromClientSide {
 
       for (int i = 0; i < 10; i++) {
         byte [] bytes = Bytes.toBytes(i);
-        get = new Get(bytes);
+        get = new Get.Builder(bytes);
         get.addFamily(FAMILIES[0]);
-        result = ht.get(get);
+        result = ht.get(get.create());
         assertTrue(result.size() == 1);
       }
       ArrayList<Delete> deletes = new ArrayList<Delete>();
@@ -1926,9 +1930,9 @@ public class TestFromClientSide {
 
       for (int i = 0; i < 10; i++) {
         byte [] bytes = Bytes.toBytes(i);
-        get = new Get(bytes);
+        get = new Get.Builder(bytes);
         get.addFamily(FAMILIES[0]);
-        result = ht.get(get);
+        result = ht.get(get.create());
         assertTrue(result.size() == 0);
       }
     }
@@ -3531,7 +3535,7 @@ public class TestFromClientSide {
     }
   }
 
-
+  
   @Test
   public void testRowsPutMultiGet() throws IOException {
     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide2.java Wed Mar 12 21:17:13 2014
@@ -100,7 +100,7 @@ public class TestFromClientSide2 {
 
     // table: row, family, c0:0, c1:1, ... , c7:7
     put = new Put(ROW);
-    for (int i=0; i < QUALIFIERS.length; i++) {
+    for (int i=0; i < QUALIFIERS.length; i++) { 
       KeyValue kv = new KeyValue(ROW, FAMILY, QUALIFIERS[i], i, VALUE);
       put.add(kv);
     }
@@ -146,14 +146,14 @@ public class TestFromClientSide2 {
     } else {
       scanner = ht.getScanner(scan);
     }
-
+    
     // First batch: c4:4, c5:5
     kvListExp = new ArrayList<KeyValue>();
     kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[4], 4, VALUE));
     kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[5], 5, VALUE));
     result = scanner.next();
     verifyResult(result, kvListExp, toLog, "Testing first batch of scan");
-
+    
     // Second batch: c6:6, c7:7
     kvListExp = new ArrayList<KeyValue>();
     kvListExp.add(new KeyValue(ROW, FAMILY, QUALIFIERS[6], 6, VALUE));
@@ -169,7 +169,7 @@ public class TestFromClientSide2 {
    * @throws Exception
    */
   @Test
-  public void testGetMaxResults() throws Exception {
+  public void testGetMaxResults() throws Exception {    
     byte [] TABLE = Bytes.toBytes("testGetMaxResults");
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 20);
@@ -182,10 +182,10 @@ public class TestFromClientSide2 {
     boolean toLog = true;
     List<KeyValue> kvListExp;
 
-    kvListExp = new ArrayList<KeyValue>();
+    kvListExp = new ArrayList<KeyValue>();  
     // Insert one CF for row[0]
     put = new Put(ROW);
-    for (int i=0; i < 10; i++) {
+    for (int i=0; i < 10; i++) { 
       KeyValue kv = new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE);
       put.add(kv);
       kvListExp.add(kv);
@@ -220,14 +220,14 @@ public class TestFromClientSide2 {
     // Insert two more CF for row[0]
     // 20 columns for CF2, 10 columns for CF1
     put = new Put(ROW);
-    for (int i=0; i < QUALIFIERS.length; i++) {
+    for (int i=0; i < QUALIFIERS.length; i++) { 
       KeyValue kv = new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE);
       put.add(kv);
     }
     ht.put(put);
 
     put = new Put(ROW);
-    for (int i=0; i < 10; i++) {
+    for (int i=0; i < 10; i++) { 
       KeyValue kv = new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE);
       put.add(kv);
     }
@@ -240,13 +240,13 @@ public class TestFromClientSide2 {
     result = ht.get(get);
     kvListExp = new ArrayList<KeyValue>();
     //Exp: CF1:q0, ..., q9, CF2: q0, q1, q10, q11, ..., q19
-    for (int i=0; i < 10; i++) {
+    for (int i=0; i < 10; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE));
     }
-    for (int i=0; i < 2; i++) {
+    for (int i=0; i < 2; i++) { 
         kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
       }
-    for (int i=10; i < 20; i++) {
+    for (int i=10; i < 20; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
     }
     verifyResult(result, kvListExp, toLog, "Testing multiple CFs");
@@ -257,13 +257,13 @@ public class TestFromClientSide2 {
     get.setFilter(new ColumnRangeFilter(QUALIFIERS[2], true, null, true));
     result = ht.get(get);
     kvListExp = new ArrayList<KeyValue>();
-    for (int i=2; i < 5; i++) {
+    for (int i=2; i < 5; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[i], 1, VALUE));
     }
-    for (int i=2; i < 5; i++) {
+    for (int i=2; i < 5; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[i], 1, VALUE));
     }
-    for (int i=2; i < 5; i++) {
+    for (int i=2; i < 5; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
     }
     verifyResult(result, kvListExp, toLog, "Testing multiple CFs + CRF");
@@ -276,7 +276,7 @@ public class TestFromClientSide2 {
     kvListExp.add(new KeyValue(ROW, FAMILIES[0], QUALIFIERS[1], 1, VALUE));
     kvListExp.add(new KeyValue(ROW, FAMILIES[1], QUALIFIERS[1], 1, VALUE));
     kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[1], 1, VALUE));
-    for (int i=10; i < 16; i++) {
+    for (int i=10; i < 16; i++) { 
       kvListExp.add(new KeyValue(ROW, FAMILIES[2], QUALIFIERS[i], 1, VALUE));
     }
     verifyResult(result, kvListExp, toLog, "Testing multiple CFs + PFF");
@@ -289,7 +289,7 @@ public class TestFromClientSide2 {
    * @throws Exception
    */
   @Test
-  public void testScanMaxResults() throws Exception {
+  public void testScanMaxResults() throws Exception {    
     byte [] TABLE = Bytes.toBytes("testScanLimit");
     byte [][] ROWS= makeNAscii(ROW, 2);
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
@@ -303,12 +303,12 @@ public class TestFromClientSide2 {
     boolean toLog = true;
     List<KeyValue> kvListExp, kvListScan;
 
-    kvListExp = new ArrayList<KeyValue>();
+    kvListExp = new ArrayList<KeyValue>();  
 
-    for (int r=0; r < ROWS.length; r++) {
+    for (int r=0; r < ROWS.length; r++) { 
       put = new Put(ROWS[r]);
-      for (int c=0; c < FAMILIES.length; c++) {
-        for (int q=0; q < QUALIFIERS.length; q++) {
+      for (int c=0; c < FAMILIES.length; c++) { 
+        for (int q=0; q < QUALIFIERS.length; q++) { 
           KeyValue kv = new KeyValue(ROWS[r], FAMILIES[c], QUALIFIERS[q], 1, VALUE);
           put.add(kv);
           if (q < 4) {
@@ -322,7 +322,7 @@ public class TestFromClientSide2 {
     scan = new Scan();
     scan.setMaxResultsPerColumnFamily(4);
     ResultScanner scanner = ht.getScanner(scan);
-    kvListScan = new ArrayList<KeyValue>();
+    kvListScan = new ArrayList<KeyValue>();      
     while ((result = scanner.next()) != null) {
       for (KeyValue kv : result.list()) {
         kvListScan.add(kv);
@@ -459,5 +459,5 @@ public class TestFromClientSide2 {
     }
     return ret;
   }
-
+  
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestGet.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestGet.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestGet.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,118 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+
+public class TestGet {
+
+  /**
+   * Tests if the object is correctly serialized & deserialized
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    ThriftCodec<Get> codec = new ThriftCodecManager()
+        .getCodec(Get.class);
+    TMemoryBuffer transport = new TMemoryBuffer(1000 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+
+    byte[] rowArray = Bytes.toBytes("myRow");
+    Map<byte[], Set<byte[]>> familyMap = new HashMap<byte[], Set<byte[]>>();
+    Set<byte[]> cols = createColumns(100);
+    for (int i = 1; i < 10; i++) {
+      familyMap.put(Bytes.toBytes("myFam" + i), cols);
+    }
+    // get with all attributes specified
+    Get get = new Get.Builder(rowArray).setMaxVersions(1)
+        .setEffectiveTS(System.currentTimeMillis()).setLockId(1L)
+        .setFamilyMap(familyMap).setStoreLimit(5)
+        .setTr(new TimeRange(0L, 100L)).create();
+    codec.write(get, protocol);
+    Get getCopy = codec.read(protocol);
+    Assert.assertEquals(get, getCopy);
+
+    // get with specifying just a row key
+    Get get2 = new Get(rowArray);
+    codec.write(get2, protocol);
+    Get getCopy2 = codec.read(protocol);
+    Assert.assertEquals(get2, getCopy2);
+
+    // test add family and add column
+    Get get3 = new Get.Builder(rowArray).addFamily(Bytes.toBytes("myFam1"))
+        .addColumn(Bytes.toBytes("myFam2"), Bytes.toBytes("col1")).create();
+    codec.write(get3, protocol);
+    Get getCopy3 = codec.read(protocol);
+    Assert.assertEquals(get3, getCopy3);
+  }
+
+  /**
+   * Returns a set of columns with prefix col
+   *
+   * @param n - the number of columns we want to obtain
+   * @return
+   */
+  public static Set<byte[]> createColumns(int n) {
+    Set<byte[]> cols = new HashSet<byte[]>();
+    String column = "col";
+    for (int i = n - 1; i >= 0; i--) {
+      cols.add(Bytes.toBytes(column + i));
+    }
+    return cols;
+  }
+
+  /**
+   * Utility method that can create n dummy gets
+   *
+   * @param n
+   * @return List of Get
+   */
+
+  public static List<Get> createDummyGets(int n) {
+    List<Get> gets = new ArrayList<>();
+    Map<byte[], Set<byte[]>> familyMap = new HashMap<byte[], Set<byte[]>>();
+    Set<byte[]> cols = createColumns(100);
+    for (int i = 1; i < 10; i++) {
+      familyMap.put(Bytes.toBytes("myFam" + i), cols);
+    }
+    for (int i = 0; i < n; i++) {
+      byte[] rowArray = Bytes.toBytes("myRow" + i);
+      Get get = new Get.Builder(rowArray).setFamilyMap(familyMap).create();
+      gets.add(get);
+    }
+    return gets;
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Wed Mar 12 21:17:13 2014
@@ -300,6 +300,7 @@ public class TestHCM {
   }
 
   private void verifyFailure(HTable table, Exception e) {
+
     List<OperationContext> context = table.getAndResetOperationContext();
     assertTrue(context.size() != 0);
     for (OperationContext c : context) {

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java Wed Mar 12 21:17:13 2014
@@ -110,7 +110,7 @@ public class TestHTable {
       Assert.assertEquals(address, address2);
     }
   }
-
+  
   @Test
   public void testHTableMultiPutThreadPool() throws Exception {
     byte [] TABLE = Bytes.toBytes("testHTableMultiputThreadPool");
@@ -122,7 +122,7 @@ public class TestHTable {
     int previousPoolSize = pool.getPoolSize();
     int previousLargestPoolSize = pool.getLargestPoolSize();
     long previousCompletedTaskCount = pool.getCompletedTaskCount();
-
+    
     for (int i = 0; i < NUM_REGIONS; i++) {
       Put put = new Put(ROWS[i]);
       put.add(FAMILY, QUALIFIER, VALUE);
@@ -135,7 +135,7 @@ public class TestHTable {
     assertEquals(previousPoolSize, pool.getPoolSize());
     assertEquals(previousLargestPoolSize, pool.getLargestPoolSize());
     assertEquals(previousCompletedTaskCount, pool.getCompletedTaskCount());
-
+    
     ArrayList<Put> multiput = new ArrayList<Put>();
     for (int i = 0; i < NUM_REGIONS; i++) {
       Put put = new Put(ROWS[i]);
@@ -144,11 +144,26 @@ public class TestHTable {
     }
     ht.put(multiput);
     ht.flushCommits();
-
+    
     // verify that HTable does use thread pool for multi put requests.
     assertTrue((SLAVES >= pool.getLargestPoolSize())
       && (pool.getLargestPoolSize() >= previousLargestPoolSize));
-    assertEquals(SLAVES,
+    assertEquals(SLAVES, 
         (pool.getCompletedTaskCount() - previousCompletedTaskCount));
   }
+
+  /**
+   * Test that when a table could not be found, a TableNotFoundException is
+   * thrown.
+   *
+   * @throws Exception
+   */
+  @Test(expected = TableNotFoundException.class)
+  public void testTableNotFound() throws Exception {
+    // Let's search for a non-existing table, and get a TableNotFoundException.
+    HConnection connection =
+      HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
+    connection.getRegionLocation(Bytes.toBytes("foo"),
+                                 Bytes.toBytes("r1"), false);
+  }
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTableClientScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTableClientScanner.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTableClientScanner.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTableClientScanner.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,63 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHTableClientScanner {
+  final Log LOG = LogFactory.getLog(getClass());
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] TABLE_NAME = Bytes.toBytes("TABLE");
+  private static final byte[] FAMILY = Bytes.toBytes("FAMILY");
+  private static final int SLAVES = 3;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(SLAVES);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testScanner() throws IOException {
+    HTable table = TEST_UTIL.createTable(TABLE_NAME, FAMILY);
+    int regionCnt = TEST_UTIL.createMultiRegions(table, FAMILY);
+    TEST_UTIL.waitUntilAllRegionsAssigned(regionCnt);
+
+    int rowCount = TEST_UTIL.loadTable(table, FAMILY);
+
+    int counted = HBaseTestingUtility.countRows(table, new Scan());
+    assertEquals("rowCount", rowCount, counted);
+  }
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestIntegerOrResultOrException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestIntegerOrResultOrException.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestIntegerOrResultOrException.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestIntegerOrResultOrException.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,77 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.IntegerOrResultOrException.Type;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+public class TestIntegerOrResultOrException {
+
+  /**
+   * This is testing the constructor of {@link IntegerOrResultOrException} which
+   * is getting Object as a parameter. This is needed in case we are reading an
+   * object which is serialized with hadoop serialization with VERSION_0 where
+   * we are serializing Object, but we are expecting to work with
+   * IntegerOrResultOrException
+   */
+  @Test
+  public void testIntegerOrResultOrExceptionObjectConstructor() {
+    // Object is an Integer
+    Object obj1 =  new Integer(1);
+    IntegerOrResultOrException value = new IntegerOrResultOrException(obj1);
+    assertEquals(value.getType(), Type.INTEGER);
+    assertEquals(obj1, value.getInteger());
+    assertEquals(null, value.getEx());
+    assertEquals(null, value.getResults());
+
+    //Object is an Exception
+    Object obj2 = new Exception("dummy exception");
+    value = new IntegerOrResultOrException(obj2);
+    assertEquals(value.getType(), Type.EXCEPTION);
+    assertEquals(((Exception) obj2).getMessage(), value.getEx().getServerJavaException().getMessage());
+    assertEquals(null, value.getInteger());
+    assertEquals(null, value.getResults());
+
+    //Object is Result[]
+    KeyValue[] kvs = new KeyValue[1];
+    KeyValue kv = new KeyValue(Bytes.toBytes("myRow"), Bytes.toBytes("myCF"),
+        Bytes.toBytes("myQualifier"), 12345L, Bytes.toBytes("myValue"));
+    kvs[0] = kv;
+    Result[] res = new Result[1];
+    res[0] = new Result(kvs);
+    Object obj3 = res;
+    value = new IntegerOrResultOrException(obj3);
+    assertEquals(value.getType(), Type.LIST_OF_RESULTS);
+    List<Result> expected = new ArrayList<Result>();
+    Collections.addAll(expected, new Result(kvs));
+    assertEquals(expected, value.getResults());
+    assertEquals(null, value.getEx());
+    assertEquals(null, value.getInteger());
+  }
+
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMaxResponseSize.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMaxResponseSize.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMaxResponseSize.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMaxResponseSize.java Wed Mar 12 21:17:13 2014
@@ -284,7 +284,7 @@ public class TestMaxResponseSize{
     String msg) {
     LOG.info(msg);
     LOG.info("Exp cnt: " + kvList.size());
-    LOG.info("True cnt is: " + result.size());	
+    LOG.info("True cnt is: " + result.size());
     assertEquals(kvList.size(), result.size());
 
     if (kvList.size() == 0) return;
@@ -293,7 +293,7 @@ public class TestMaxResponseSize{
       KeyValue kvExp = kvList.get(i++);
       if (toLog) {
         LOG.info("get kv is: " + kv.toString());
-        LOG.info("exp kv is: " + kvExp.toString());	  
+        LOG.info("exp kv is: " + kvExp.toString());
       }
       assertTrue("Not equal", kvExp.equals(kv));
     }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java Wed Mar 12 21:17:13 2014
@@ -63,34 +63,34 @@ public class TestMetaScanner {
           Bytes.toBytes("region_b")});
     // Make sure all the regions are deployed
     TEST_UTIL.countRows(table);
-
-    MetaScanner.MetaScannerVisitor visitor =
+    
+    MetaScanner.MetaScannerVisitor visitor = 
       mock(MetaScanner.MetaScannerVisitor.class);
     doReturn(true).when(visitor).processRow((Result)anyObject());
 
     // Scanning the entire table should give us three rows
     MetaScanner.metaScan(conf, visitor, TABLENAME);
     verify(visitor, times(3)).processRow((Result)anyObject());
-
+    
     // Scanning the table with a specified empty start row should also
     // give us three META rows
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, HConstants.EMPTY_BYTE_ARRAY, 1000);
     verify(visitor, times(3)).processRow((Result)anyObject());
-
+    
     // Scanning the table starting in the middle should give us two rows:
     // region_a and region_b
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1000);
     verify(visitor, times(2)).processRow((Result)anyObject());
-
+    
     // Scanning with a limit of 1 should only give us one row
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1);
     verify(visitor, times(1)).processRow((Result)anyObject());
-
+        
   }
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiAction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiAction.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiAction.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiAction.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,56 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+public class TestMultiAction {
+  /**
+   * Test if the MultiResponse is correctly serialized and deserialized
+   * @throws Exception
+   */
+  @Test
+  public void testThriftSerializeDeserialze() throws Exception {
+    // create a map of dummy byte array which maps to list of dummy gets
+    Map<byte[], List<Get>> gets = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    gets.put(new byte[]{1,2,3}, TestGet.createDummyGets(7));
+
+    // create a map of dummy byte array which maps to list of dummy puts
+    Map<byte[], List<Put>> puts = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    puts.put(new byte[]{7,  8, 9}, TestPut.createDummyPuts(2));
+
+    // create a map of dummy byte array which maps to list of dummy deletes
+    Map<byte[], List<Delete>> deletes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    deletes.put(new byte[] {10, 11, 12}, TestDelete.createDummyDeletes(4));
+
+    //construct the MultiAction object out of the previously created maps.
+    MultiAction multiAction = new MultiAction(gets, puts, deletes);
+    MultiAction multiActionCopy = Bytes.readThriftBytes(Bytes.writeThriftBytes(multiAction, MultiAction.class), MultiAction.class);
+
+    Assert.assertEquals(multiAction, multiActionCopy);
+  }
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPut.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPut.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPut.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPut.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2013 The Apache Software Foundation
+ *
+ * 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.client;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static junit.framework.Assert.assertEquals;
+
+public class TestMultiPut {
+  /**
+   * Test the serialization and deserialization of the MultiPut object using
+   * Swift.
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    ThriftCodec<MultiPut> codec =
+      new ThriftCodecManager().getCodec(MultiPut.class);
+    TMemoryBuffer transport = new TMemoryBuffer(100*1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+
+    Map<byte[], List<Put>> puts = createDummyMultiPut();
+    MultiPut multiPut = new MultiPut(puts);
+    codec.write(multiPut, protocol);
+    MultiPut multiPutCopy = codec.read(protocol);
+    assertEquals(multiPut, multiPutCopy);
+  }
+
+  Map<byte[], List<Put>> createDummyMultiPut() {
+    Map<byte[], List<Put>> puts =
+      new TreeMap<byte[], List<Put>>(Bytes.BYTES_COMPARATOR);
+    for (int i = 0; i < 10; i++) {
+      byte[] regionName = Bytes.toBytes("region" + i);
+      List<Put> putList = createDummyPuts(10);
+      puts.put(regionName, putList);
+    }
+    return puts;
+  }
+
+  List<Put> createDummyPuts(int n) {
+    List<Put> putList = new ArrayList<Put>();
+    for (int i = 0; i < n; i++) {
+      Put p = new Put(Bytes.toBytes("row" + i), System.currentTimeMillis());
+      p.add(Bytes.toBytes("cf"), Bytes.toBytes("q"), Bytes.toBytes("myValue"));
+      putList.add(p);
+    }
+    return putList;
+  }
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPutResponse.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPutResponse.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPutResponse.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultiPutResponse.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,53 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Test;
+
+public class TestMultiPutResponse {
+
+  /**
+   * Test if the MultiPutResponse object is correctly serialized and
+   * deserialized.
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    byte[][] r = { Bytes.toBytes("r1"), Bytes.toBytes("r2"),
+        Bytes.toBytes("r3") };
+    MultiPutResponse m = new MultiPutResponse();
+    for (int i = 0; i < r.length; i++) {
+      m.addResult(r[i], i);
+    }
+    ThriftCodec<MultiPutResponse> codec = new ThriftCodecManager().
+        getCodec(MultiPutResponse.class);
+    TMemoryBuffer transport = new TMemoryBuffer(1000 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+    codec.write(m, protocol);
+
+    MultiPutResponse other = codec.read(protocol);
+    Assert.assertEquals(m, other);
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java Wed Mar 12 21:17:13 2014
@@ -513,3 +513,4 @@ public class TestMultipleTimestamps {
     ht.delete(del);
   }
 }
+

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java Wed Mar 12 21:17:13 2014
@@ -60,7 +60,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.codehaus.jackson.map.ObjectMapper;
 
 /**
- * Run tests that use the funtionality of the Operation superclass for
+ * Run tests that use the funtionality of the Operation superclass for 
  * Puts, Gets, Deletes, Scans, and MultiPuts.
  */
 public class TestOperation {
@@ -283,7 +283,7 @@ public class TestOperation {
   }
 
   /**
-   * Test the client Operations' JSON encoding to ensure that produced JSON is
+   * Test the client Operations' JSON encoding to ensure that produced JSON is 
    * parseable and that the details are present and not corrupted.
    * @throws IOException
    */
@@ -344,7 +344,7 @@ public class TestOperation {
     assertEquals("Qualifier incorrect in Put.toJSON()",
         Bytes.toStringBinary(QUALIFIER),
         kvMap.get("qualifier"));
-    assertEquals("Value length incorrect in Put.toJSON()",
+    assertEquals("Value length incorrect in Put.toJSON()", 
         VALUE.length, kvMap.get("vlen"));
 
     // produce a Delete operation
@@ -362,7 +362,7 @@ public class TestOperation {
     assertNotNull("Family absent in Delete.toJSON()", familyInfo);
     assertEquals("KeyValue absent in Delete.toJSON()", 1, familyInfo.size());
     kvMap = (Map) familyInfo.get(0);
-    assertEquals("Qualifier incorrect in Delete.toJSON()",
+    assertEquals("Qualifier incorrect in Delete.toJSON()", 
         Bytes.toStringBinary(QUALIFIER), kvMap.get("qualifier"));
     
     // produce a RowMutations operation

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestParallelScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestParallelScanner.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestParallelScanner.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestParallelScanner.java Wed Mar 12 21:17:13 2014
@@ -19,6 +19,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -28,12 +34,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
 public class TestParallelScanner {
   final Log LOG = LogFactory.getLog(getClass());
   private final static HBaseTestingUtility TEST_UTIL =
@@ -59,7 +59,6 @@ public class TestParallelScanner {
     final int regionCnt = TEST_UTIL.createMultiRegions(table, FAMILY);
     TEST_UTIL.waitUntilAllRegionsAssigned(regionCnt);
     final int rowCount = TEST_UTIL.loadTable(table, FAMILY);
-    table.flushCommits();
     TEST_UTIL.flush(name);
 
     Scan[] rowsPerRegionScanner = new Scan[regionCnt];
@@ -75,9 +74,11 @@ public class TestParallelScanner {
       s.setStopRow(startAndEndKeys.getSecond()[i]);
       s.addFamily(FAMILY);
       rowsPerRegionScanner[i] = s;
-      totalRowsScannedInSequential += TEST_UTIL.countRows(table, s);
+      totalRowsScannedInSequential += HBaseTestingUtility.countRows(table,
+          new Scan(s));
     }
-    assertEquals(rowCount, totalRowsScannedInSequential);
+    assertEquals("Total rows scanned in sequential", rowCount,
+        totalRowsScannedInSequential);
 
     // Construct a ParallelScanner
     ParallelScanner parallelScanner =
@@ -90,6 +91,7 @@ public class TestParallelScanner {
       totalRowScannedInParallel += results.size();
     }
     parallelScanner.close();
-    assertEquals(rowCount, totalRowScannedInParallel);
+    assertEquals("Total row scanned in parallel", rowCount,
+        totalRowScannedInParallel);
   }
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestPut.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestPut.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestPut.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestPut.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Test;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+
+public class TestPut {
+
+  /**
+   * Test if the serialization and deserialization of Put works when using
+   * Swift.
+   * @throws Exception
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+
+    ThriftCodec<Put> codec = new ThriftCodecManager().getCodec(Put.class);
+    TMemoryBuffer transport = new TMemoryBuffer(1000 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+
+    Map<byte[], List<KeyValue>> fMap =
+      new TreeMap<byte[], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+    byte[] row = Bytes.toBytes("row");
+    fMap.put(row, createDummyKVs(row));
+    long lockId = 5;
+    boolean writeToWAL = true;
+    Put put = new Put(row, System.currentTimeMillis(), fMap, lockId,
+        writeToWAL);
+
+    codec.write(put, protocol);
+    Put putCopy = codec.read(protocol);
+    assertEquals(put, putCopy);
+  }
+
+  @Test
+  public void testIsDummy() throws Exception {
+    Assert.assertTrue("Put.isDummy", new Put().isDummy());
+    Assert.assertTrue("Put.isDummy", new Put(new byte[0]).isDummy());
+  }
+
+  public static List<KeyValue> createDummyKVs(byte[] row) {
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    for (int i = 0; i < 10; i++) {
+      byte[] family = Bytes.toBytes("fam" + i);
+      byte[] qualifier = Bytes.toBytes("q" + i);
+      byte[] value = Bytes.toBytes("v" + i);
+      kvs.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(),
+        KeyValue.Type.Delete, value));
+    }
+    return kvs;
+  }
+
+  /**
+   * Test if the Builder is functionally equivalent to the constructor.
+   * @throws Exception
+   */
+  @Test
+  public void testBuilder() throws Exception {
+    byte[] row = Bytes.toBytes("row");
+    Map<byte[], List<KeyValue>> familyMap =
+      new TreeMap<byte[], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
+    familyMap.put(row, createDummyKVs(row));
+    long currentMs = System.currentTimeMillis();
+    long lockId = 5;
+    boolean writeToWAL = true;
+    // Create a Put the standard way
+    Put put1 = new Put(row, currentMs, familyMap, lockId, writeToWAL);
+
+    // Now use a builder to create one, with the same parameters.
+    Put put2 = new Put.Builder()
+      .setRow(row)
+      .setFamilyMap(familyMap)
+      .setTimeStamp(currentMs)
+      .setLockId(lockId)
+      .setWriteToWAL(writeToWAL)
+      .create();
+
+    // Check if they match.
+    Assert.assertEquals(put1, put2);
+  }
+
+  /**
+   * Utility method that can create n dummy puts
+   *
+   * @param n
+   * @return
+   */
+  public static List<Put> createDummyPuts(int n) {
+    List<Put> puts = new ArrayList<Put>();
+    String row = "row";
+    for (int i = 0; i < n; i++) {
+      Map<byte[], List<KeyValue>> fMap = new TreeMap<byte[], List<KeyValue>>(
+          Bytes.BYTES_COMPARATOR);
+      byte[] rowBytes = Bytes.toBytes(row + i);
+      fMap.put(rowBytes, createDummyKVs(rowBytes));
+      Put put = new Put.Builder().setRow(rowBytes).setFamilyMap(fMap)
+          .setTimeStamp(System.currentTimeMillis()).setLockId(1)
+          .setWriteToWAL(false).create();
+      puts.add(put);
+    }
+    return puts;
+  }
+
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestResult.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestResult.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestResult.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,73 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+
+
+public class TestResult {
+
+  /**
+   * Tests if the Result Object is correctly serialized & deserialized
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    ThriftCodec<Result> codec = new ThriftCodecManager().getCodec(Result.class);
+    TMemoryBuffer transport = new TMemoryBuffer(1000 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+
+    List<KeyValue> kvs = constuctKvList(100);
+    Result res = new Result(kvs);
+
+    codec.write(res, protocol);
+    Result resCopy = codec.read(protocol);
+    Assert.assertEquals(res, resCopy);
+  }
+
+  /**
+   * Returns a dummy list of KeyValue
+   *
+   * @param n - the number of kvs in the resulting list
+   * @return
+   */
+  public List<KeyValue> constuctKvList(int n) {
+    List<KeyValue> list = new ArrayList<KeyValue>();
+    for (int i = 0; i < n; i++) {
+      KeyValue kv = new KeyValue(Bytes.toBytes("myRow" + i),
+          Bytes.toBytes("myCF"), Bytes.toBytes("myQualifier"), 12345L,
+          Bytes.toBytes("myValue"));
+      list.add(kv);
+    }
+    return list;
+  }
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestTMultiResponse.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestTMultiResponse.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestTMultiResponse.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestTMultiResponse.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,63 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.client;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+public class TestTMultiResponse {
+
+  /**
+   * Test if the {@link TMultiResponse} is correctly serialized and de-serialized
+   * @throws Exception
+   */
+  @Test
+  public void testThriftSerializeDeserialze() throws Exception {
+    List<Map<byte[], IntegerOrResultOrException>> maps = new ArrayList<>();
+    // create 3 dummy maps that should imitate the containers inside
+    // MultiResponse: resultsForGet, resultsForPut and resultsForDelete
+    for (int i = 0; i < 3; i++) {
+      Map<byte[], IntegerOrResultOrException> map = new TreeMap<>(
+          Bytes.BYTES_COMPARATOR);
+      map.put(new byte[] { 1, 2, 3 }, new IntegerOrResultOrException(1));
+      List<Result> results = new ArrayList<>();
+      results.add(new Result(TestPut.createDummyKVs(Bytes.toBytes("row"))));
+      map.put(new byte[] { 7, 8, 9 }, new IntegerOrResultOrException(results));
+      map.put(new byte[] { 4, 5, 6 }, new IntegerOrResultOrException(
+          new RuntimeException("runtime exception happened")));
+      maps.add(map);
+    }
+    // construct the MultiResponse Object out of those maps
+    TMultiResponse response = new TMultiResponse(maps.get(0), maps.get(1),
+        maps.get(2));
+    TMultiResponse responseCopy = Bytes.readThriftBytes(
+        Bytes.writeThriftBytes(response, TMultiResponse.class),
+        TMultiResponse.class);
+    Assert.assertEquals(response, responseCopy);
+  }
+
+}