You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2013/10/31 21:30:36 UTC

svn commit: r1537626 - in /hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client: TestRemoteAdminRetries.java TestRemoteHTableRetries.java TestRemoteTable.java

Author: enis
Date: Thu Oct 31 20:30:35 2013
New Revision: 1537626

URL: http://svn.apache.org/r1537626
Log:
HBASE-8543 fix coverage org.apache.hadoop.hbase.rest.client

Added:
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java
Modified:
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java?rev=1537626&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java Thu Oct 31 20:30:35 2013
@@ -0,0 +1,164 @@
+/*
+ * 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.rest.client;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests {@link RemoteAdmin} retries.
+ */
+@Category(SmallTests.class)
+public class TestRemoteAdminRetries {
+
+  private static final int SLEEP_TIME = 50;
+  private static final int RETRIES = 3;
+  private static final long MAX_TIME = SLEEP_TIME * (RETRIES - 1);
+  
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  
+  private RemoteAdmin remoteAdmin;
+  private Client client;
+
+  @Before
+  public void setup() throws Exception {
+    client = mock(Client.class);
+    Response response = new Response(509);
+    when(client.get(anyString(), anyString())).thenReturn(response);
+    when(client.delete(anyString())).thenReturn(response);
+    when(client.put(anyString(), anyString(), any(byte[].class))).thenReturn(response);
+    when(client.post(anyString(), anyString(), any(byte[].class))).thenReturn(response);
+    Configuration configuration = TEST_UTIL.getConfiguration();
+
+    configuration.setInt("hbase.rest.client.max.retries", RETRIES);
+    configuration.setInt("hbase.rest.client.sleep", SLEEP_TIME);
+
+    remoteAdmin = new RemoteAdmin(client, TEST_UTIL.getConfiguration(), "MyTable");
+  }
+
+  @Test
+  public void testFailingGetRestVersion() throws Exception  {
+    testTimedOutGetCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.getRestVersion();
+      }
+    });
+  }
+  
+  @Test
+  public void testFailingGetClusterStatus() throws Exception  {
+    testTimedOutGetCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.getClusterStatus();
+      }
+    });
+  }
+
+  @Test
+  public void testFailingGetClusterVersion() throws Exception {
+    testTimedOutGetCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.getClusterVersion();
+      }
+    });
+  }
+
+  @Test
+  public void testFailingGetTableAvailable() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.isTableAvailable(Bytes.toBytes("TestTable"));
+      }
+    });
+  }
+
+  @Test
+  public void testFailingCreateTable() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.createTable(new HTableDescriptor(Bytes.toBytes("TestTable")));
+      }
+    });
+    verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));
+  }
+
+  @Test
+  public void testFailingDeleteTable() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.deleteTable("TestTable");
+      }
+    });
+    verify(client, times(RETRIES)).delete(anyString());
+  }
+
+  @Test
+  public void testFailingGetTableList() throws Exception {
+    testTimedOutGetCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteAdmin.getTableList();
+      }
+    });
+  }
+  
+  private void testTimedOutGetCall(CallExecutor callExecutor) throws Exception {
+    testTimedOutCall(callExecutor);
+    verify(client, times(RETRIES)).get(anyString(), anyString());
+  }
+  
+  private void testTimedOutCall(CallExecutor callExecutor) throws Exception {
+    long start = System.currentTimeMillis();
+    try {
+      callExecutor.run();
+      fail("should be timeout exception!");
+    } catch (IOException e) {
+      assertTrue(Pattern.matches(".*MyTable.*timed out", e.toString()));
+    }
+    assertTrue((System.currentTimeMillis() - start) > MAX_TIME);
+  }
+
+  private static interface CallExecutor {
+    void run() throws Exception;
+  }
+  
+}

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java?rev=1537626&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteHTableRetries.java Thu Oct 31 20:30:35 2013
@@ -0,0 +1,193 @@
+/*
+ * 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.rest.client;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.SmallTests;
+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.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test RemoteHTable retries.
+ */
+@Category(SmallTests.class)
+public class TestRemoteHTableRetries {
+
+  private static final int SLEEP_TIME = 50;
+  private static final int RETRIES = 3;
+  private static final long MAX_TIME = SLEEP_TIME * (RETRIES - 1);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  
+  private static final byte[] ROW_1 = Bytes.toBytes("testrow1");
+  private static final byte[] COLUMN_1 = Bytes.toBytes("a");
+  private static final byte[] QUALIFIER_1 = Bytes.toBytes("1");
+  private static final byte[] VALUE_1 = Bytes.toBytes("testvalue1");
+
+  private Client client;
+  private RemoteHTable remoteTable;
+  
+  @Before
+  public void setup() throws Exception {
+    client = mock(Client.class);
+    Response response = new Response(509);
+    when(client.get(anyString(), anyString())).thenReturn(response);
+    when(client.delete(anyString())).thenReturn(response);
+    when(client.put(anyString(), anyString(), any(byte[].class))).thenReturn(
+        response);
+    when(client.post(anyString(), anyString(), any(byte[].class))).thenReturn(
+        response);
+
+    Configuration configuration = TEST_UTIL.getConfiguration();
+    configuration.setInt("hbase.rest.client.max.retries", RETRIES);
+    configuration.setInt("hbase.rest.client.sleep", SLEEP_TIME);
+
+    remoteTable = new RemoteHTable(client, TEST_UTIL.getConfiguration(),
+        "MyTable");
+  }
+
+  @After
+  public void tearDownAfterClass() throws Exception {
+    remoteTable.close();
+  }
+  
+  @Test
+  public void testDelete() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        Delete delete = new Delete(Bytes.toBytes("delete"));
+        remoteTable.delete(delete);
+      }
+    });
+    verify(client, times(RETRIES)).delete(anyString());
+  }
+  
+  @Test
+  public void testGet() throws Exception {
+    testTimedOutGetCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteTable.get(new Get(Bytes.toBytes("Get")));
+      }
+    });
+  }
+
+  @Test
+  public void testSingleRowPut() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteTable.put(new Put(Bytes.toBytes("Row")));
+      }
+    });
+    verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));
+  }
+  
+  @Test
+  public void testMultiRowPut() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        Put[] puts = { new Put(Bytes.toBytes("Row1")),
+            new Put(Bytes.toBytes("Row2")) };
+        remoteTable.put(Arrays.asList(puts));
+      }
+    });
+    verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));
+  }
+
+  @Test
+  public void testGetScanner() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        remoteTable.getScanner(new Scan());
+      }
+    });
+    verify(client, times(RETRIES)).post(anyString(), anyString(), any(byte[].class));
+  }
+  
+  @Test
+  public void testCheckAndPut() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        Put put = new Put(ROW_1);
+        put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+        remoteTable.checkAndPut(ROW_1, COLUMN_1, QUALIFIER_1, VALUE_1, put );
+      }
+    });
+    verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));
+  }
+
+  @Test
+  public void testCheckAndDelete() throws Exception {
+    testTimedOutCall(new CallExecutor() {
+      @Override
+      public void run() throws Exception {
+        Put put = new Put(ROW_1);
+        put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+        Delete delete= new Delete(ROW_1);
+        remoteTable.checkAndDelete(ROW_1, COLUMN_1, QUALIFIER_1,  VALUE_1, delete );
+      }
+    });
+  }
+  
+  private void testTimedOutGetCall(CallExecutor callExecutor) throws Exception {
+    testTimedOutCall(callExecutor);
+    verify(client, times(RETRIES)).get(anyString(), anyString());
+  }
+  
+  private void testTimedOutCall(CallExecutor callExecutor) throws Exception {
+    long start = System.currentTimeMillis();
+    try {
+      callExecutor.run();
+      fail("should be timeout exception!");
+    } catch (IOException e) {
+      assertTrue(Pattern.matches(".*request timed out", e.toString()));
+    }
+    assertTrue((System.currentTimeMillis() - start) > MAX_TIME);
+  }
+
+  private static interface CallExecutor {
+    void run() throws Exception;
+  }
+
+}

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java?rev=1537626&r1=1537625&r2=1537626&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java Thu Oct 31 20:30:35 2013
@@ -20,13 +20,24 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
+import org.apache.commons.httpclient.Header;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -36,20 +47,16 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.rest.HBaseRESTTestingUtility;
-import org.apache.hadoop.hbase.rest.client.Client;
-import org.apache.hadoop.hbase.rest.client.Cluster;
-import org.apache.hadoop.hbase.rest.client.RemoteHTable;
 import org.apache.hadoop.hbase.util.Bytes;
-
-import static org.junit.Assert.*;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category(MediumTests.class)
 public class TestRemoteTable {
-  private static final Log LOG = LogFactory.getLog(TestRemoteTable.class);
   private static final String TABLE = "TestRemoteTable";
   private static final byte[] ROW_1 = Bytes.toBytes("testrow1");
   private static final byte[] ROW_2 = Bytes.toBytes("testrow2");
@@ -70,24 +77,29 @@ public class TestRemoteTable {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final HBaseRESTTestingUtility REST_TEST_UTIL = 
     new HBaseRESTTestingUtility();
-  private static RemoteHTable remoteTable;
+  private RemoteHTable remoteTable;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
+  }
+
+  @Before
+  public void before() throws Exception  {
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    LOG.info("Admin Connection=" + admin.getConnection() + ", " + 
-      admin.getConnection().getZooKeeperWatcher());
-    if (!admin.tableExists(TABLE)) {
-      HTableDescriptor htd = new HTableDescriptor(TABLE);
-      htd.addFamily(new HColumnDescriptor(COLUMN_1));
-      htd.addFamily(new HColumnDescriptor(COLUMN_2));
-      htd.addFamily(new HColumnDescriptor(COLUMN_3));
-      admin.createTable(htd);
-      HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
-      LOG.info("Table connection=" + table.getConnection() + ", " +
-        admin.getConnection().getZooKeeperWatcher());
+    if (admin.tableExists(TABLE)) {
+      if (admin.isTableEnabled(TABLE)) admin.disableTable(TABLE);
+      admin.deleteTable(TABLE);
+    }
+    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    htd.addFamily(new HColumnDescriptor(COLUMN_1).setMaxVersions(3));
+    htd.addFamily(new HColumnDescriptor(COLUMN_2).setMaxVersions(3));
+    htd.addFamily(new HColumnDescriptor(COLUMN_3).setMaxVersions(3));
+    admin.createTable(htd);
+    HTable table = null;
+    try {
+      table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
       Put put = new Put(ROW_1);
       put.add(COLUMN_1, QUALIFIER_1, TS_2, VALUE_1);
       table.put(put);
@@ -97,25 +109,36 @@ public class TestRemoteTable {
       put.add(COLUMN_2, QUALIFIER_2, TS_2, VALUE_2);
       table.put(put);
       table.flushCommits();
+    } finally {
+      if (null != table) table.close();
     }
     remoteTable = new RemoteHTable(
       new Client(new Cluster().add("localhost", 
           REST_TEST_UTIL.getServletPort())),
         TEST_UTIL.getConfiguration(), TABLE);
   }
-
+  
+  @After
+  public void after() throws Exception {
+    remoteTable.close();
+  }
+  
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    remoteTable.close();
     REST_TEST_UTIL.shutdownServletContainer();
     TEST_UTIL.shutdownMiniCluster();
   }
 
   @Test
   public void testGetTableDescriptor() throws IOException {
-    HTableDescriptor local = new HTable(TEST_UTIL.getConfiguration(),
-      TABLE).getTableDescriptor();
-    assertEquals(remoteTable.getTableDescriptor(), local);
+    HTable table = null;
+    try {
+      table = new HTable(TEST_UTIL.getConfiguration(), TABLE);
+      HTableDescriptor local = table.getTableDescriptor();
+      assertEquals(remoteTable.getTableDescriptor(), local);
+    } finally {
+      if (null != table) table.close();
+    }
   }
 
   @Test
@@ -301,6 +324,8 @@ public class TestRemoteTable {
     value = result.getValue(COLUMN_2, QUALIFIER_2);
     assertNotNull(value);
     assertTrue(Bytes.equals(VALUE_2, value));
+
+    assertTrue(Bytes.equals(Bytes.toBytes("TestRemoteTable"), remoteTable.getTableName()));
   }
 
   @Test
@@ -362,6 +387,7 @@ public class TestRemoteTable {
     assertNull(value2);
   }
 
+  @Test
   public void testScanner() throws IOException {
     List<Put> puts = new ArrayList<Put>();
     Put put = new Put(ROW_1);
@@ -385,21 +411,123 @@ public class TestRemoteTable {
     assertEquals(1, results.length);
     assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
 
-    results = scanner.next(3);
+    Result result = scanner.next();
+    assertNotNull(result);
+    assertTrue(Bytes.equals(ROW_2, result.getRow()));
+
+    results = scanner.next(2);
     assertNotNull(results);
-    assertEquals(3, results.length);
-    assertTrue(Bytes.equals(ROW_2, results[0].getRow()));
-    assertTrue(Bytes.equals(ROW_3, results[1].getRow()));
-    assertTrue(Bytes.equals(ROW_4, results[2].getRow()));
+    assertEquals(2, results.length);
+    assertTrue(Bytes.equals(ROW_3, results[0].getRow()));
+    assertTrue(Bytes.equals(ROW_4, results[1].getRow()));
 
     results = scanner.next(1);
     assertNull(results);
+    scanner.close();
 
+    scanner = remoteTable.getScanner(COLUMN_1);
+    results = scanner.next(4);
+    assertNotNull(results);
+    assertEquals(4, results.length);
+    assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
+    assertTrue(Bytes.equals(ROW_2, results[1].getRow()));
+    assertTrue(Bytes.equals(ROW_3, results[2].getRow()));
+    assertTrue(Bytes.equals(ROW_4, results[3].getRow()));
+
+    scanner.close();
+
+    scanner = remoteTable.getScanner(COLUMN_1, QUALIFIER_1);
+    results = scanner.next(4);
+    assertNotNull(results);
+    assertEquals(4, results.length);
+    assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
+    assertTrue(Bytes.equals(ROW_2, results[1].getRow()));
+    assertTrue(Bytes.equals(ROW_3, results[2].getRow()));
+    assertTrue(Bytes.equals(ROW_4, results[3].getRow()));
     scanner.close();
+    assertTrue(remoteTable.isAutoFlush());
+  }
+
+  @Test
+  public void testCheckAndDelete() throws IOException {
+    Get get = new Get(ROW_1);
+    Result result = remoteTable.get(get);
+    byte[] value1 = result.getValue(COLUMN_1, QUALIFIER_1);
+    byte[] value2 = result.getValue(COLUMN_2, QUALIFIER_2);
+    assertNotNull(value1);
+    assertTrue(Bytes.equals(VALUE_1, value1));
+    assertNull(value2);
+    assertTrue(remoteTable.exists(get));
+    assertEquals(1, remoteTable.get(Collections.singletonList(get)).length);
+    Delete delete = new Delete(ROW_1);
+
+    remoteTable.checkAndDelete(ROW_1, COLUMN_1, QUALIFIER_1, VALUE_1, delete);
+    assertFalse(remoteTable.exists(get));
+
+    Put put = new Put(ROW_1);
+    put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+    remoteTable.put(put);
+
+    assertTrue(remoteTable.checkAndPut(ROW_1, COLUMN_1, QUALIFIER_1, VALUE_1, put));
+    assertFalse(remoteTable.checkAndPut(ROW_1, COLUMN_1, QUALIFIER_1, VALUE_2, put));
+  }
+
+  /**
+   * Test RemoteHable.Scanner.iterator method  
+   */
+  @Test
+  public void testIteratorScaner() throws IOException {
+    List<Put> puts = new ArrayList<Put>();
+    Put put = new Put(ROW_1);
+    put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+    puts.add(put);
+    put = new Put(ROW_2);
+    put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+    puts.add(put);
+    put = new Put(ROW_3);
+    put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+    puts.add(put);
+    put = new Put(ROW_4);
+    put.add(COLUMN_1, QUALIFIER_1, VALUE_1);
+    puts.add(put);
+    remoteTable.put(puts);
+
+    ResultScanner scanner = remoteTable.getScanner(new Scan());
+    Iterator<Result> iterator = scanner.iterator();
+    assertTrue(iterator.hasNext());
+    int counter = 0;
+    while (iterator.hasNext()) {
+      iterator.next();
+      counter++;
+    }
+    assertEquals(4, counter);
+  }
+
+  /**
+   * Test a some methods of class Response.
+   */
+  @Test
+  public void testResponse(){
+    Response response = new Response(200);
+    assertEquals(200, response.getCode());
+    Header[] headers = new Header[2];
+    headers[0] = new Header("header1", "value1");
+    headers[1] = new Header("header2", "value2");
+    response = new Response(200, headers);
+    assertEquals("value1", response.getHeader("header1"));
+    assertFalse(response.hasBody());
+    response.setCode(404);
+    assertEquals(404, response.getCode());
+    headers = new Header[2];
+    headers[0] = new Header("header1", "value1.1");
+    headers[1] = new Header("header2", "value2");
+    response.setHeaders(headers);
+    assertEquals("value1.1", response.getHeader("header1"));
+    response.setBody(Bytes.toBytes("body"));
+    assertTrue(response.hasBody());    
   }
 
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
-}
-
+}
\ No newline at end of file