001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.replication;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertTrue;
022import java.io.IOException;
023import java.util.EnumSet;
024import java.util.List;
025import org.apache.hadoop.hbase.ClusterMetrics;
026import org.apache.hadoop.hbase.ClusterMetrics.Option;
027import org.apache.hadoop.hbase.HBaseClassTestRule;
028import org.apache.hadoop.hbase.ServerMetrics;
029import org.apache.hadoop.hbase.ServerName;
030import org.apache.hadoop.hbase.Waiter;
031import org.apache.hadoop.hbase.client.Admin;
032import org.apache.hadoop.hbase.client.Put;
033import org.apache.hadoop.hbase.regionserver.HRegionServer;
034import org.apache.hadoop.hbase.testclassification.MediumTests;
035import org.apache.hadoop.hbase.testclassification.ReplicationTests;
036import org.apache.hadoop.hbase.util.Bytes;
037import org.apache.hadoop.hbase.util.JVMClusterUtil;
038import org.apache.hadoop.hbase.util.Threads;
039import org.junit.ClassRule;
040import org.junit.Test;
041import org.junit.experimental.categories.Category;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045@Category({ ReplicationTests.class, MediumTests.class })
046public class TestReplicationStatus extends TestReplicationBase {
047  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStatus.class);
048
049  @ClassRule
050  public static final HBaseClassTestRule CLASS_RULE =
051    HBaseClassTestRule.forClass(TestReplicationStatus.class);
052
053  static void insertRowsOnSource() throws IOException {
054    final byte[] qualName = Bytes.toBytes("q");
055    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
056      Put p = new Put(Bytes.toBytes("row" + i));
057      p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
058      htable1.put(p);
059    }
060  }
061
062  /**
063   * Test for HBASE-9531.
064   * <p/>
065   * put a few rows into htable1, which should be replicated to htable2 <br/>
066   * create a ClusterStatus instance 'status' from HBaseAdmin <br/>
067   * test : status.getLoad(server).getReplicationLoadSourceList() <br/>
068   * test : status.getLoad(server).getReplicationLoadSink()
069   */
070  @Test
071  public void testReplicationStatus() throws Exception {
072    // This test wants two RS's up. We only run one generally so add one.
073    UTIL1.getMiniHBaseCluster().startRegionServer();
074    Waiter.waitFor(UTIL1.getConfiguration(), 30000, new Waiter.Predicate<Exception>() {
075      @Override public boolean evaluate() throws Exception {
076        return UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads().size() > 1;
077      }
078    });
079    Admin hbaseAdmin = UTIL1.getAdmin();
080    // disable peer <= WHY? I DON'T GET THIS DISABLE BUT TEST FAILS W/O IT.
081    hbaseAdmin.disableReplicationPeer(PEER_ID2);
082    insertRowsOnSource();
083    LOG.info("AFTER PUTS");
084    // TODO: Change this wait to a barrier. I tried waiting on replication stats to
085    // change but sleeping in main thread seems to mess up background replication.
086    // HACK! To address flakeyness.
087    Threads.sleep(10000);
088    ClusterMetrics metrics = hbaseAdmin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
089    for (JVMClusterUtil.RegionServerThread thread : UTIL1.getHBaseCluster().
090        getRegionServerThreads()) {
091      ServerName server = thread.getRegionServer().getServerName();
092      assertTrue("" + server, metrics.getLiveServerMetrics().containsKey(server));
093      ServerMetrics sm = metrics.getLiveServerMetrics().get(server);
094      List<ReplicationLoadSource> rLoadSourceList = sm.getReplicationLoadSourceList();
095      ReplicationLoadSink rLoadSink = sm.getReplicationLoadSink();
096
097      // check SourceList only has one entry, because only has one peer
098      assertEquals("Failed to get ReplicationLoadSourceList " +
099        rLoadSourceList + ", " + server,1, rLoadSourceList.size());
100      assertEquals(PEER_ID2, rLoadSourceList.get(0).getPeerID());
101
102      // check Sink exist only as it is difficult to verify the value on the fly
103      assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
104        (rLoadSink.getAgeOfLastAppliedOp() >= 0));
105      assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
106        (rLoadSink.getTimestampsOfLastAppliedOp() >= 0));
107    }
108
109    // Stop rs1, then the queue of rs1 will be transfered to rs0
110    HRegionServer hrs = UTIL1.getHBaseCluster().getRegionServer(1);
111    hrs.stop("Stop RegionServer");
112    while(!hrs.isShutDown()) {
113      Threads.sleep(100);
114    }
115    // To be sure it dead and references cleaned up. TODO: Change this to a barrier.
116    // I tried waiting on replication stats to change but sleeping in main thread
117    // seems to mess up background replication.
118    Threads.sleep(10000);
119    ServerName server = UTIL1.getHBaseCluster().getRegionServer(0).getServerName();
120    List<ReplicationLoadSource> rLoadSourceList = waitOnMetricsReport(1, server);
121    // The remaining server should now have two queues -- the original and then the one that was
122    // added because of failover. The original should still be PEER_ID2 though.
123    assertEquals("Failed ReplicationLoadSourceList " + rLoadSourceList, 2, rLoadSourceList.size());
124    assertEquals(PEER_ID2, rLoadSourceList.get(0).getPeerID());
125  }
126
127  /**
128   * Wait until Master shows metrics counts for ReplicationLoadSourceList that are
129   * greater than <code>greaterThan</code> for <code>serverName</code> before
130   * returning. We want to avoid case where RS hasn't yet updated Master before
131   * allowing test proceed.
132   * @param greaterThan size of replicationLoadSourceList must be greater before we proceed
133   */
134  private List<ReplicationLoadSource> waitOnMetricsReport(int greaterThan, ServerName serverName)
135      throws IOException {
136    ClusterMetrics metrics = hbaseAdmin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS));
137    List<ReplicationLoadSource> list =
138      metrics.getLiveServerMetrics().get(serverName).getReplicationLoadSourceList();
139    while(list.size() <= greaterThan) {
140      Threads.sleep(1000);
141    }
142    return list;
143  }
144}