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.fail;
022
023import java.util.concurrent.ExecutionException;
024import org.apache.hadoop.hbase.HBaseClassTestRule;
025import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
026import org.apache.hadoop.hbase.client.AsyncConnection;
027import org.apache.hadoop.hbase.client.AsyncTable;
028import org.apache.hadoop.hbase.client.ConnectionFactory;
029import org.apache.hadoop.hbase.client.Get;
030import org.apache.hadoop.hbase.client.Put;
031import org.apache.hadoop.hbase.client.RegionInfoBuilder;
032import org.apache.hadoop.hbase.client.Table;
033import org.apache.hadoop.hbase.regionserver.HRegionServer;
034import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
035import org.apache.hadoop.hbase.testclassification.LargeTests;
036import org.apache.hadoop.hbase.testclassification.ReplicationTests;
037import org.apache.hadoop.hbase.util.Bytes;
038import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
039import org.junit.BeforeClass;
040import org.junit.ClassRule;
041import org.junit.Test;
042import org.junit.experimental.categories.Category;
043import org.slf4j.Logger;
044import org.slf4j.LoggerFactory;
045
046@Category({ ReplicationTests.class, LargeTests.class })
047public class TestSyncReplicationMoreLogsInLocalCopyToRemote extends SyncReplicationTestBase {
048
049  @ClassRule
050  public static final HBaseClassTestRule CLASS_RULE =
051    HBaseClassTestRule.forClass(TestSyncReplicationMoreLogsInLocalCopyToRemote.class);
052
053  private static final Logger LOG =
054    LoggerFactory.getLogger(TestSyncReplicationMoreLogsInLocalCopyToRemote.class);
055
056  @BeforeClass
057  public static void setUp() throws Exception {
058    UTIL1.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
059      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
060    UTIL2.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL,
061      DualAsyncFSWALForTest.class, DualAsyncFSWAL.class);
062    SyncReplicationTestBase.setUp();
063  }
064
065  @Test
066  public void testSplitLog() throws Exception {
067    UTIL1.getAdmin().disableReplicationPeer(PEER_ID);
068    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
069      SyncReplicationState.STANDBY);
070    UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
071      SyncReplicationState.ACTIVE);
072    HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME);
073    DualAsyncFSWALForTest wal =
074      (DualAsyncFSWALForTest) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
075    wal.setRemoteBroken();
076    try (AsyncConnection conn =
077      ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) {
078      AsyncTable<?> table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1).build();
079      try {
080        table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0))).get();
081        fail("Should fail since the rs will crash and we will not retry");
082      } catch (ExecutionException e) {
083        // expected
084        LOG.info("Expected error:", e);
085      }
086    }
087    UTIL1.waitFor(60000, new ExplainingPredicate<Exception>() {
088
089      @Override
090      public boolean evaluate() throws Exception {
091        try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) {
092          return table.exists(new Get(Bytes.toBytes(0)));
093        }
094      }
095
096      @Override
097      public String explainFailure() throws Exception {
098        return "The row is still not available";
099      }
100    });
101    UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID,
102      SyncReplicationState.DOWNGRADE_ACTIVE);
103    // We should have copied the local log to remote, so we should be able to get the value
104    try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) {
105      assertEquals(0, Bytes.toInt(table.get(new Get(Bytes.toBytes(0))).getValue(CF, CQ)));
106    }
107  }
108}