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;
019
020import static org.junit.Assert.assertEquals;
021
022import java.io.IOException;
023import java.util.List;
024import java.util.concurrent.ExecutionException;
025import java.util.concurrent.TimeUnit;
026import java.util.concurrent.TimeoutException;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hbase.client.Admin;
029import org.apache.hadoop.hbase.client.Put;
030import org.apache.hadoop.hbase.client.RegionInfo;
031import org.apache.hadoop.hbase.client.RegionLocator;
032import org.apache.hadoop.hbase.client.Table;
033import org.apache.hadoop.hbase.regionserver.HRegion;
034import org.apache.hadoop.hbase.regionserver.HRegionServer;
035import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
036import org.apache.hadoop.hbase.testclassification.LargeTests;
037import org.apache.hadoop.hbase.testclassification.RegionServerTests;
038import org.apache.hadoop.hbase.util.Bytes;
039import org.apache.hadoop.hbase.wal.NoEOFWALStreamReader;
040import org.apache.hadoop.hbase.wal.WAL;
041import org.apache.hadoop.hbase.wal.WALStreamReader;
042import org.junit.After;
043import org.junit.AfterClass;
044import org.junit.BeforeClass;
045import org.junit.ClassRule;
046import org.junit.Test;
047import org.junit.experimental.categories.Category;
048
049/**
050 * Testcase for HBASE-20066
051 */
052@Category({ RegionServerTests.class, LargeTests.class })
053public class TestSequenceIdMonotonicallyIncreasing {
054
055  @ClassRule
056  public static final HBaseClassTestRule CLASS_RULE =
057    HBaseClassTestRule.forClass(TestSequenceIdMonotonicallyIncreasing.class);
058
059  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
060
061  private static final TableName NAME = TableName.valueOf("test");
062
063  private static final byte[] CF = Bytes.toBytes("cf");
064
065  private static final byte[] CQ = Bytes.toBytes("cq");
066
067  @BeforeClass
068  public static void setUpBeforeClass() throws Exception {
069    UTIL.startMiniCluster(2);
070  }
071
072  @AfterClass
073  public static void tearDownAfterClass() throws Exception {
074    UTIL.shutdownMiniCluster();
075  }
076
077  @After
078  public void tearDown() throws IOException {
079    Admin admin = UTIL.getAdmin();
080    if (admin.tableExists(NAME)) {
081      admin.disableTable(NAME);
082      admin.deleteTable(NAME);
083    }
084  }
085
086  private Table createTable(boolean multiRegions) throws IOException {
087    if (multiRegions) {
088      return UTIL.createTable(NAME, CF, new byte[][] { Bytes.toBytes(1) });
089    } else {
090      return UTIL.createTable(NAME, CF);
091    }
092  }
093
094  private long getMaxSeqId(HRegionServer rs, RegionInfo region) throws IOException {
095    Path walFile = ((AbstractFSWAL<?>) rs.getWAL(null)).getCurrentFileName();
096    long maxSeqId = -1L;
097    try (WALStreamReader reader =
098      NoEOFWALStreamReader.create(UTIL.getTestFileSystem(), walFile, UTIL.getConfiguration())) {
099      for (;;) {
100        WAL.Entry entry = reader.next();
101        if (entry == null) {
102          break;
103        }
104        if (Bytes.equals(region.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName())) {
105          maxSeqId = Math.max(maxSeqId, entry.getKey().getSequenceId());
106        }
107      }
108    }
109    return maxSeqId;
110  }
111
112  @Test
113  public void testSplit()
114    throws IOException, InterruptedException, ExecutionException, TimeoutException {
115    try (Table table = createTable(false)) {
116      table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0)));
117      table.put(new Put(Bytes.toBytes(1)).addColumn(CF, CQ, Bytes.toBytes(0)));
118    }
119    UTIL.flush(NAME);
120    HRegionServer rs = UTIL.getRSForFirstRegionInTable(NAME);
121    RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo();
122    UTIL.getAdmin().splitRegionAsync(region.getRegionName(), Bytes.toBytes(1)).get(1,
123      TimeUnit.MINUTES);
124    long maxSeqId = getMaxSeqId(rs, region);
125    RegionLocator locator = UTIL.getConnection().getRegionLocator(NAME);
126    HRegionLocation locA = locator.getRegionLocation(Bytes.toBytes(0), true);
127    HRegionLocation locB = locator.getRegionLocation(Bytes.toBytes(1), true);
128    assertEquals(maxSeqId + 1, locA.getSeqNum());
129    assertEquals(maxSeqId + 1, locB.getSeqNum());
130  }
131
132  @Test
133  public void testMerge()
134    throws IOException, InterruptedException, ExecutionException, TimeoutException {
135    try (Table table = createTable(true)) {
136      table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0)));
137      table.put(new Put(Bytes.toBytes(1)).addColumn(CF, CQ, Bytes.toBytes(0)));
138      table.put(new Put(Bytes.toBytes(2)).addColumn(CF, CQ, Bytes.toBytes(0)));
139    }
140    UTIL.flush(NAME);
141    SingleProcessHBaseCluster cluster = UTIL.getMiniHBaseCluster();
142    List<HRegion> regions = cluster.getRegions(NAME);
143    HRegion regionA = regions.get(0);
144    HRegion regionB = regions.get(1);
145    HRegionServer rsA =
146      cluster.getRegionServer(cluster.getServerWith(regionA.getRegionInfo().getRegionName()));
147    HRegionServer rsB =
148      cluster.getRegionServer(cluster.getServerWith(regionB.getRegionInfo().getRegionName()));
149    UTIL.getAdmin().mergeRegionsAsync(regionA.getRegionInfo().getRegionName(),
150      regionB.getRegionInfo().getRegionName(), false).get(1, TimeUnit.MINUTES);
151    long maxSeqIdA = getMaxSeqId(rsA, regionA.getRegionInfo());
152    long maxSeqIdB = getMaxSeqId(rsB, regionB.getRegionInfo());
153    HRegionLocation loc =
154      UTIL.getConnection().getRegionLocator(NAME).getRegionLocation(Bytes.toBytes(0), true);
155    assertEquals(Math.max(maxSeqIdA, maxSeqIdB) + 1, loc.getSeqNum());
156  }
157}