Repository: hbase
Updated Branches:
  refs/heads/master 287f95a57 -> 805d39fca


http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
deleted file mode 100644
index 5bd2ff1..0000000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransaction.java
+++ /dev/null
@@ -1,485 +0,0 @@
-/**
- * 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.regionserver;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.*;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-import com.google.common.collect.ImmutableList;
-
-/**
- * Test the {@link RegionMergeTransactionImpl} class against two HRegions (as
- * opposed to running cluster).
- */
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestRegionMergeTransaction {
-  private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final Path testdir = TEST_UTIL.getDataTestDir(this.getClass()
-      .getName());
-  private HRegion region_a;
-  private HRegion region_b;
-  private HRegion region_c;
-  private WALFactory wals;
-  private FileSystem fs;
-  // Start rows of region_a,region_b,region_c
-  private static final byte[] STARTROW_A = new byte[] { 'a', 'a', 'a' };
-  private static final byte[] STARTROW_B = new byte[] { 'g', 'g', 'g' };
-  private static final byte[] STARTROW_C = new byte[] { 'w', 'w', 'w' };
-  private static final byte[] ENDROW = new byte[] { '{', '{', '{' };
-  private static final byte[] CF = HConstants.CATALOG_FAMILY;
-
-  @Before
-  public void setup() throws IOException {
-    this.fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    this.fs.delete(this.testdir, true);
-    final Configuration walConf = new 
Configuration(TEST_UTIL.getConfiguration());
-    FSUtils.setRootDir(walConf, this.testdir);
-    this.wals = new WALFactory(walConf, null, 
TestRegionMergeTransaction.class.getName());
-    this.region_a = createRegion(this.testdir, this.wals, STARTROW_A, 
STARTROW_B);
-    this.region_b = createRegion(this.testdir, this.wals, STARTROW_B, 
STARTROW_C);
-    this.region_c = createRegion(this.testdir, this.wals, STARTROW_C, ENDROW);
-    assert region_a != null && region_b != null && region_c != null;
-    TEST_UTIL.getConfiguration().setBoolean("hbase.testing.nocluster", true);
-  }
-
-  @After
-  public void teardown() throws IOException {
-    for (HRegion region : new HRegion[] { region_a, region_b, region_c }) {
-      if (region != null && !region.isClosed()) region.close();
-      if (this.fs.exists(region.getRegionFileSystem().getRegionDir())
-          && !this.fs.delete(region.getRegionFileSystem().getRegionDir(), 
true)) {
-        throw new IOException("Failed deleting of "
-            + region.getRegionFileSystem().getRegionDir());
-      }
-    }
-    if (this.wals != null) {
-      this.wals.close();
-    }
-    this.fs.delete(this.testdir, true);
-  }
-
-  /**
-   * Test straight prepare works. Tries to merge on {@link #region_a} and
-   * {@link #region_b}
-   * @throws IOException
-   */
-  @Test
-  public void testPrepare() throws IOException {
-    prepareOnGoodRegions();
-  }
-
-  private RegionMergeTransactionImpl prepareOnGoodRegions() throws IOException 
{
-    RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, 
region_b,
-        false);
-    RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_a.getRegionInfo().getRegionName());
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_b.getRegionInfo().getRegionName());
-    assertTrue(spyMT.prepare(null));
-    return spyMT;
-  }
-
-  /**
-   * Test merging the same region
-   */
-  @Test
-  public void testPrepareWithSameRegion() throws IOException {
-    RegionMergeTransactionImpl mt = new 
RegionMergeTransactionImpl(this.region_a,
-        this.region_a, true);
-    assertFalse("should not merge the same region even if it is forcible ",
-        mt.prepare(null));
-  }
-
-  /**
-   * Test merging two not adjacent regions under a common merge
-   */
-  @Test
-  public void testPrepareWithRegionsNotAdjacent() throws IOException {
-    RegionMergeTransactionImpl mt = new 
RegionMergeTransactionImpl(this.region_a,
-        this.region_c, false);
-    assertFalse("should not merge two regions if they are adjacent except it 
is forcible",
-        mt.prepare(null));
-  }
-
-  /**
-   * Test merging two not adjacent regions under a compulsory merge
-   */
-  @Test
-  public void testPrepareWithRegionsNotAdjacentUnderCompulsory()
-      throws IOException {
-    RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, 
region_c,
-        true);
-    RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_a.getRegionInfo().getRegionName());
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_c.getRegionInfo().getRegionName());
-    assertTrue("Since focible is true, should merge two regions even if they 
are not adjacent",
-        spyMT.prepare(null));
-  }
-
-  /**
-   * Pass a reference store
-   */
-  @Test
-  public void testPrepareWithRegionsWithReference() throws IOException {
-    HStore storeMock = Mockito.mock(HStore.class);
-    when(storeMock.hasReferences()).thenReturn(true);
-    when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
-    when(storeMock.getSizeToFlush()).thenReturn(new MemstoreSize());
-    when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
-    this.region_a.stores.put(Bytes.toBytes(""), storeMock);
-    RegionMergeTransactionImpl mt = new 
RegionMergeTransactionImpl(this.region_a,
-        this.region_b, false);
-    assertFalse(
-        "a region should not be mergeable if it has instances of store file 
references",
-        mt.prepare(null));
-  }
-
-  @Test
-  public void testPrepareWithClosedRegion() throws IOException {
-    this.region_a.close();
-    RegionMergeTransactionImpl mt = new 
RegionMergeTransactionImpl(this.region_a,
-        this.region_b, false);
-    assertFalse(mt.prepare(null));
-  }
-
-  /**
-   * Test merging regions which are merged regions and has reference in 
hbase:meta all
-   * the same
-   */
-  @Test
-  public void testPrepareWithRegionsWithMergeReference() throws IOException {
-    RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, 
region_b,
-        false);
-    RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
-    doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
-        region_a.getRegionInfo().getRegionName());
-    doReturn(true).when(spyMT).hasMergeQualifierInMeta(null,
-        region_b.getRegionInfo().getRegionName());
-    assertFalse(spyMT.prepare(null));
-  }
-
-  /**
-   * Test RegionMergeTransactionListener
-   */
-  @Test public void testRegionMergeTransactionListener() throws IOException {
-    RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a, 
region_b,
-        false);
-    RegionMergeTransactionImpl spyMT = Mockito.spy(mt);
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_a.getRegionInfo().getRegionName());
-    doReturn(false).when(spyMT).hasMergeQualifierInMeta(null,
-        region_b.getRegionInfo().getRegionName());
-    RegionMergeTransaction.TransactionListener listener =
-            Mockito.mock(RegionMergeTransaction.TransactionListener.class);
-    mt.registerTransactionListener(listener);
-    mt.prepare(null);
-    TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
-    CoordinatedStateManager cp = 
CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      TEST_UTIL.getConfiguration());
-    Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
-    mt.execute(mockServer, null);
-    verify(listener).transition(mt,
-            RegionMergeTransaction.RegionMergeTransactionPhase.STARTED,
-            RegionMergeTransaction.RegionMergeTransactionPhase.PREPARED);
-    verify(listener, times(10)).transition(any(RegionMergeTransaction.class),
-            any(RegionMergeTransaction.RegionMergeTransactionPhase.class),
-            any(RegionMergeTransaction.RegionMergeTransactionPhase.class));
-    verifyNoMoreInteractions(listener);
-  }
-
-  @Test
-  public void testWholesomeMerge() throws IOException, InterruptedException {
-    final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
-    final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
-    assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
-    assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
-    assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
-
-    // Start transaction.
-    RegionMergeTransactionImpl mt = prepareOnGoodRegions();
-
-    // Run the execute. Look at what it returns.
-    TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
-    CoordinatedStateManager cp = 
CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      TEST_UTIL.getConfiguration());
-    Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
-    HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
-    // Do some assertions about execution.
-    assertTrue(this.fs.exists(mt.getMergesDir()));
-    // Assert region_a and region_b is closed.
-    assertTrue(region_a.isClosed());
-    assertTrue(region_b.isClosed());
-
-    // Assert mergedir is empty -- because its content will have been moved out
-    // to be under the merged region dirs.
-    assertEquals(0, this.fs.listStatus(mt.getMergesDir()).length);
-    // Check merged region have correct key span.
-    assertTrue(Bytes.equals(this.region_a.getRegionInfo().getStartKey(),
-        mergedRegion.getRegionInfo().getStartKey()));
-    assertTrue(Bytes.equals(this.region_b.getRegionInfo().getEndKey(),
-        mergedRegion.getRegionInfo().getEndKey()));
-    // Count rows. merged region are already open
-    try {
-      int mergedRegionRowCount = TEST_UTIL.countRows(mergedRegion);
-      assertEquals((rowCountOfRegionA + rowCountOfRegionB),
-          mergedRegionRowCount);
-    } finally {
-      HBaseTestingUtility.closeRegionAndWAL(mergedRegion);
-    }
-    // Assert the write lock is no longer held on region_a and region_b
-    assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
-    assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
-  }
-
-  @Test
-  public void testRollback() throws IOException, InterruptedException {
-    final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
-    final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
-    assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
-    assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
-    assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
-
-    // Start transaction.
-    RegionMergeTransactionImpl mt = prepareOnGoodRegions();
-
-    when(mt.createMergedRegionFromMerges(region_a, region_b,
-        mt.getMergedRegionInfo())).thenThrow(
-        new MockedFailedMergedRegionCreation());
-
-    // Run the execute. Look at what it returns.
-    boolean expectedException = false;
-    TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
-    CoordinatedStateManager cp = 
CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      TEST_UTIL.getConfiguration());
-    Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
-    try {
-      mt.execute(mockServer, null);
-    } catch (MockedFailedMergedRegionCreation e) {
-      expectedException = true;
-    }
-    assertTrue(expectedException);
-    // Run rollback
-    assertTrue(mt.rollback(null, null));
-
-    // Assert I can scan region_a and region_b.
-    int rowCountOfRegionA2 = TEST_UTIL.countRows(this.region_a);
-    assertEquals(rowCountOfRegionA, rowCountOfRegionA2);
-    int rowCountOfRegionB2 = TEST_UTIL.countRows(this.region_b);
-    assertEquals(rowCountOfRegionB, rowCountOfRegionB2);
-
-    // Assert rollback cleaned up stuff in fs
-    assertTrue(!this.fs.exists(HRegion.getRegionDir(this.testdir,
-        mt.getMergedRegionInfo())));
-
-    assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
-    assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
-
-    // Now retry the merge but do not throw an exception this time.
-    assertTrue(mt.prepare(null));
-    HRegion mergedRegion = (HRegion)mt.execute(mockServer, null);
-    // Count rows. daughters are already open
-    // Count rows. merged region are already open
-    try {
-      int mergedRegionRowCount = TEST_UTIL.countRows(mergedRegion);
-      assertEquals((rowCountOfRegionA + rowCountOfRegionB),
-          mergedRegionRowCount);
-    } finally {
-      HBaseTestingUtility.closeRegionAndWAL(mergedRegion);
-    }
-    // Assert the write lock is no longer held on region_a and region_b
-    assertTrue(!this.region_a.lock.writeLock().isHeldByCurrentThread());
-    assertTrue(!this.region_b.lock.writeLock().isHeldByCurrentThread());
-  }
-
-  @Test
-  public void testFailAfterPONR() throws IOException, KeeperException, 
InterruptedException {
-    final int rowCountOfRegionA = loadRegion(this.region_a, CF, true);
-    final int rowCountOfRegionB = loadRegion(this.region_b, CF, true);
-    assertTrue(rowCountOfRegionA > 0 && rowCountOfRegionB > 0);
-    assertEquals(rowCountOfRegionA, TEST_UTIL.countRows(this.region_a));
-    assertEquals(rowCountOfRegionB, TEST_UTIL.countRows(this.region_b));
-
-    // Start transaction.
-    RegionMergeTransactionImpl mt = prepareOnGoodRegions();
-    Mockito.doThrow(new MockedFailedMergedRegionOpen())
-        .when(mt)
-        .openMergedRegion((Server) Mockito.anyObject(),
-            (RegionServerServices) Mockito.anyObject(),
-            (HRegion) Mockito.anyObject());
-
-    // Run the execute. Look at what it returns.
-    boolean expectedException = false;
-    TEST_UTIL.getConfiguration().setInt(HConstants.REGIONSERVER_PORT, 0);
-    CoordinatedStateManager cp = 
CoordinatedStateManagerFactory.getCoordinatedStateManager(
-      TEST_UTIL.getConfiguration());
-    Server mockServer = new HRegionServer(TEST_UTIL.getConfiguration(), cp);
-    try {
-      mt.execute(mockServer, null);
-    } catch (MockedFailedMergedRegionOpen e) {
-      expectedException = true;
-    }
-    assertTrue(expectedException);
-    // Run rollback returns false that we should restart.
-    assertFalse(mt.rollback(null, null));
-    // Make sure that merged region is still in the filesystem, that
-    // they have not been removed; this is supposed to be the case if we go
-    // past point of no return.
-    Path tableDir = this.region_a.getRegionFileSystem().getRegionDir()
-        .getParent();
-    Path mergedRegionDir = new Path(tableDir, mt.getMergedRegionInfo()
-        .getEncodedName());
-    assertTrue(TEST_UTIL.getTestFileSystem().exists(mergedRegionDir));
-  }
-
-  @Test
-  public void testMergedRegionBoundary() {
-    TableName tableName =
-        TableName.valueOf("testMergedRegionBoundary");
-    byte[] a = Bytes.toBytes("a");
-    byte[] b = Bytes.toBytes("b");
-    byte[] z = Bytes.toBytes("z");
-    HRegionInfo r1 = new HRegionInfo(tableName);
-    HRegionInfo r2 = new HRegionInfo(tableName, a, z);
-    HRegionInfo m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
-    assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
-        && Bytes.equals(m.getEndKey(), r1.getEndKey()));
-
-    r1 = new HRegionInfo(tableName, null, a);
-    r2 = new HRegionInfo(tableName, a, z);
-    m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
-    assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
-        && Bytes.equals(m.getEndKey(), r2.getEndKey()));
-
-    r1 = new HRegionInfo(tableName, null, a);
-    r2 = new HRegionInfo(tableName, z, null);
-    m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
-    assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
-        && Bytes.equals(m.getEndKey(), r2.getEndKey()));
-
-    r1 = new HRegionInfo(tableName, a, z);
-    r2 = new HRegionInfo(tableName, z, null);
-    m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
-    assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
-      && Bytes.equals(m.getEndKey(), r2.getEndKey()));
-
-    r1 = new HRegionInfo(tableName, a, b);
-    r2 = new HRegionInfo(tableName, b, z);
-    m = RegionMergeTransactionImpl.getMergedRegionInfo(r1, r2);
-    assertTrue(Bytes.equals(m.getStartKey(), r1.getStartKey())
-      && Bytes.equals(m.getEndKey(), r2.getEndKey()));
-  }
-
-  /**
-   * Exception used in this class only.
-   */
-  @SuppressWarnings("serial")
-  private class MockedFailedMergedRegionCreation extends IOException {
-  }
-
-  @SuppressWarnings("serial")
-  private class MockedFailedMergedRegionOpen extends IOException {
-  }
-
-  private HRegion createRegion(final Path testdir, final WALFactory wals,
-      final byte[] startrow, final byte[] endrow)
-      throws IOException {
-    // Make a region with start and end keys.
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("table"));
-    HColumnDescriptor hcd = new HColumnDescriptor(CF);
-    htd.addFamily(hcd);
-    HRegionInfo hri = new HRegionInfo(htd.getTableName(), startrow, endrow);
-    HRegion a = HBaseTestingUtility.createRegionAndWAL(hri, testdir,
-        TEST_UTIL.getConfiguration(), htd);
-    HBaseTestingUtility.closeRegionAndWAL(a);
-    return HRegion.openHRegion(testdir, hri, htd,
-      wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()),
-      TEST_UTIL.getConfiguration());
-  }
-
-  /**
-   * Load region with rows from 'aaa' to 'zzz', skip the rows which are out of
-   * range of the region
-   * @param r Region
-   * @param f Family
-   * @param flush flush the cache if true
-   * @return Count of rows loaded.
-   * @throws IOException
-   */
-  private int loadRegion(final HRegion r, final byte[] f, final boolean flush)
-      throws IOException {
-    byte[] k = new byte[3];
-    int rowCount = 0;
-    for (byte b1 = 'a'; b1 <= 'z'; b1++) {
-      for (byte b2 = 'a'; b2 <= 'z'; b2++) {
-        for (byte b3 = 'a'; b3 <= 'z'; b3++) {
-          k[0] = b1;
-          k[1] = b2;
-          k[2] = b3;
-          if (!HRegion.rowIsInRange(r.getRegionInfo(), k)) {
-            continue;
-          }
-          Put put = new Put(k);
-          put.addColumn(f, null, k);
-          if (r.getWAL() == null)
-            put.setDurability(Durability.SKIP_WAL);
-          r.put(put);
-          rowCount++;
-        }
-      }
-      if (flush) {
-        r.flush(true);
-      }
-    }
-    return rowCount;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
deleted file mode 100644
index 661af14..0000000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- *
- * 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.util;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Tests merging a normal table's regions
- */
-@Category({MiscTests.class, MediumTests.class})
-public class TestMergeTable {
-  private static final Log LOG = LogFactory.getLog(TestMergeTable.class);
-  private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static final byte [] COLUMN_NAME = Bytes.toBytes("contents");
-  private static final byte [] VALUE;
-  static {
-    // We will use the same value for the rows as that is not really important 
here
-    String partialValue = String.valueOf(System.currentTimeMillis());
-    StringBuilder val = new StringBuilder();
-    while (val.length() < 1024) {
-      val.append(partialValue);
-    }
-    VALUE = Bytes.toBytes(val.toString());
-  }
-
-  /**
-   * Test merge.
-   * Hand-makes regions of a mergeable size and adds the hand-made regions to
-   * hand-made meta.  The hand-made regions are created offline.  We then start
-   * up mini cluster, disables the hand-made table and starts in on merging.
-   * @throws Exception
-   */
-  @Test (timeout=300000) public void testMergeTable() throws Exception {
-    // Table we are manually creating offline.
-    HTableDescriptor desc = new 
HTableDescriptor(org.apache.hadoop.hbase.TableName.valueOf(Bytes.toBytes("test")));
-    desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
-
-    // Set maximum regionsize down.
-    UTIL.getConfiguration().setLong(HConstants.HREGION_MAX_FILESIZE, 64L * 
1024L * 1024L);
-    // Make it so we don't split.
-    UTIL.getConfiguration().setInt("hbase.regionserver.regionSplitLimit", 0);
-    // Startup hdfs.  Its in here we'll be putting our manually made regions.
-    UTIL.startMiniDFSCluster(1);
-    // Create hdfs hbase rootdir.
-    Path rootdir = UTIL.createRootDir();
-    FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    if (fs.exists(rootdir)) {
-      if (fs.delete(rootdir, true)) {
-        LOG.info("Cleaned up existing " + rootdir);
-      }
-    }
-
-    // Now create three data regions: The first is too large to merge since it
-    // will be > 64 MB in size. The second two will be smaller and will be
-    // selected for merging.
-
-    // To ensure that the first region is larger than 64MB we need to write at
-    // least 65536 rows. We will make certain by writing 70000
-    byte [] row_70001 = Bytes.toBytes("row_70001");
-    byte [] row_80001 = Bytes.toBytes("row_80001");
-
-    // Create regions and populate them at same time.  Create the tabledir
-    // for them first.
-    new FSTableDescriptors(UTIL.getConfiguration(), fs, 
rootdir).createTableDescriptor(desc);
-    HRegion [] regions = {
-      createRegion(desc, null, row_70001, 1, 70000, rootdir),
-      createRegion(desc, row_70001, row_80001, 70001, 10000, rootdir),
-      createRegion(desc, row_80001, null, 80001, 11000, rootdir)
-    };
-
-    // Now create the root and meta regions and insert the data regions
-    // created above into hbase:meta
-    setupMeta(rootdir, regions);
-    try {
-      LOG.info("Starting mini zk cluster");
-      UTIL.startMiniZKCluster();
-      LOG.info("Starting mini hbase cluster");
-      UTIL.startMiniHBaseCluster(1, 1);
-      Configuration c = new Configuration(UTIL.getConfiguration());
-      Connection connection = UTIL.getConnection();
-
-      List<HRegionInfo> originalTableRegions =
-        MetaTableAccessor.getTableRegions(connection, desc.getTableName());
-      LOG.info("originalTableRegions size=" + originalTableRegions.size() +
-        "; " + originalTableRegions);
-      Admin admin = connection.getAdmin();
-      admin.disableTable(desc.getTableName());
-      admin.close();
-      HMerge.merge(c, FileSystem.get(c), desc.getTableName());
-      List<HRegionInfo> postMergeTableRegions =
-        MetaTableAccessor.getTableRegions(connection, desc.getTableName());
-      LOG.info("postMergeTableRegions size=" + postMergeTableRegions.size() +
-        "; " + postMergeTableRegions);
-      assertTrue("originalTableRegions=" + originalTableRegions.size() +
-        ", postMergeTableRegions=" + postMergeTableRegions.size(),
-        postMergeTableRegions.size() < originalTableRegions.size());
-      LOG.info("Done with merge");
-    } finally {
-      UTIL.shutdownMiniCluster();
-      LOG.info("After cluster shutdown");
-    }
-  }
-
-  private HRegion createRegion(final HTableDescriptor desc,
-      byte [] startKey, byte [] endKey, int firstRow, int nrows, Path rootdir)
-  throws IOException {
-    HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey);
-    HRegion region = HBaseTestingUtility.createRegionAndWAL(hri, rootdir, 
UTIL.getConfiguration(),
-        desc);
-    LOG.info("Created region " + 
region.getRegionInfo().getRegionNameAsString());
-    for(int i = firstRow; i < firstRow + nrows; i++) {
-      Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
-      put.setDurability(Durability.SKIP_WAL);
-      put.addColumn(COLUMN_NAME, null, VALUE);
-      region.put(put);
-      if (i % 10000 == 0) {
-        LOG.info("Flushing write #" + i);
-        region.flush(true);
-      }
-    }
-    HBaseTestingUtility.closeRegionAndWAL(region);
-    return region;
-  }
-
-  protected void setupMeta(Path rootdir, final HRegion [] regions)
-  throws IOException {
-    HRegion meta =
-      
HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO, 
rootdir,
-          UTIL.getConfiguration(), UTIL.getMetaTableDescriptor());
-    for (HRegion r: regions) {
-      HRegion.addRegionToMETA(meta, r);
-    }
-    HBaseTestingUtility.closeRegionAndWAL(meta);
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/805d39fc/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
deleted file mode 100644
index 1924c9e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/**
- *
- * 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.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/** Test stand alone merge tool that can merge arbitrary regions */
-@Category({MiscTests.class, LargeTests.class})
-public class TestMergeTool extends HBaseTestCase {
-  private static final Log LOG = LogFactory.getLog(TestMergeTool.class);
-  HBaseTestingUtility TEST_UTIL;
-//  static final byte [] COLUMN_NAME = Bytes.toBytes("contents:");
-  static final byte [] FAMILY = Bytes.toBytes("contents");
-  static final byte [] QUALIFIER = Bytes.toBytes("dc");
-
-  private final HRegionInfo[] sourceRegions = new HRegionInfo[5];
-  private final HRegion[] regions = new HRegion[5];
-  private HTableDescriptor desc;
-  private byte [][][] rows;
-  private MiniDFSCluster dfsCluster = null;
-  private WALFactory wals;
-
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    // Set the timeout down else this test will take a while to complete.
-    this.conf.setLong("hbase.zookeeper.recoverable.waittime", 10);
-    // Make it so we try and connect to a zk that is not there (else we might
-    // find a zk ensemble put up by another concurrent test and this will
-    // mess up this test.  Choose unlikely port. Default test port is 21818.
-    // Default zk port is 2181.
-    this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 10001);
-
-    this.conf.set("hbase.hstore.compactionThreshold", "2");
-
-    // Create table description
-    this.desc = new 
HTableDescriptor(org.apache.hadoop.hbase.TableName.valueOf("TestMergeTool"));
-    this.desc.addFamily(new HColumnDescriptor(FAMILY));
-
-    /*
-     * Create the HRegionInfos for the regions.
-     */
-    // Region 0 will contain the key range [row_0200,row_0300)
-    sourceRegions[0] = new HRegionInfo(this.desc.getTableName(),
-        Bytes.toBytes("row_0200"),
-      Bytes.toBytes("row_0300"));
-
-    // Region 1 will contain the key range [row_0250,row_0400) and overlaps
-    // with Region 0
-    sourceRegions[1] =
-      new HRegionInfo(this.desc.getTableName(),
-          Bytes.toBytes("row_0250"),
-          Bytes.toBytes("row_0400"));
-
-    // Region 2 will contain the key range [row_0100,row_0200) and is adjacent
-    // to Region 0 or the region resulting from the merge of Regions 0 and 1
-    sourceRegions[2] =
-      new HRegionInfo(this.desc.getTableName(),
-          Bytes.toBytes("row_0100"),
-          Bytes.toBytes("row_0200"));
-
-    // Region 3 will contain the key range [row_0500,row_0600) and is not
-    // adjacent to any of Regions 0, 1, 2 or the merged result of any or all
-    // of those regions
-    sourceRegions[3] =
-      new HRegionInfo(this.desc.getTableName(),
-          Bytes.toBytes("row_0500"),
-          Bytes.toBytes("row_0600"));
-
-    // Region 4 will have empty start and end keys and overlaps all regions.
-    sourceRegions[4] =
-      new HRegionInfo(this.desc.getTableName(),
-          HConstants.EMPTY_BYTE_ARRAY,
-          HConstants.EMPTY_BYTE_ARRAY);
-
-    /*
-     * Now create some row keys
-     */
-    this.rows = new byte [5][][];
-    this.rows[0] = Bytes.toByteArrays(new String[] { "row_0210", "row_0280" });
-    this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350",
-        "row_035" });
-    this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175",
-        "row_0175", "row_0175"});
-    this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560",
-        "row_0560", "row_0560", "row_0560"});
-    this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000",
-        "row_1000", "row_1000", "row_1000", "row_1000" });
-
-    // Start up dfs
-    TEST_UTIL = new HBaseTestingUtility(conf);
-    this.dfsCluster = TEST_UTIL.startMiniDFSCluster(2);
-    this.fs = this.dfsCluster.getFileSystem();
-    System.out.println("fs=" + this.fs);
-    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
-    TEST_UTIL.createRootDir();
-
-    // Note: we must call super.setUp after starting the mini cluster or
-    // we will end up with a local file system
-
-    super.setUp();
-    wals = new WALFactory(conf, null, "TestMergeTool");
-    try {
-      // Create meta region
-      createMetaRegion();
-      new FSTableDescriptors(this.conf, this.fs, 
testDir).createTableDescriptor(this.desc);
-      /*
-       * Create the regions we will merge
-       */
-      for (int i = 0; i < sourceRegions.length; i++) {
-        regions[i] =
-          HBaseTestingUtility.createRegionAndWAL(this.sourceRegions[i], 
testDir, this.conf,
-              this.desc);
-        /*
-         * Insert data
-         */
-        for (int j = 0; j < rows[i].length; j++) {
-          byte [] row = rows[i][j];
-          Put put = new Put(row);
-          put.addColumn(FAMILY, QUALIFIER, row);
-          regions[i].put(put);
-        }
-        HRegion.addRegionToMETA(meta, regions[i]);
-      }
-      // Close root and meta regions
-      closeRootAndMeta();
-
-    } catch (Exception e) {
-      TEST_UTIL.shutdownMiniCluster();
-      throw e;
-    }
-  }
-
-  @Override
-  @After
-  public void tearDown() throws Exception {
-    super.tearDown();
-    for (int i = 0; i < sourceRegions.length; i++) {
-      HRegion r = regions[i];
-      if (r != null) {
-        HBaseTestingUtility.closeRegionAndWAL(r);
-      }
-    }
-    wals.close();
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /*
-   * @param msg Message that describes this merge
-   * @param regionName1
-   * @param regionName2
-   * @param log Log to use merging.
-   * @param upperbound Verifying, how high up in this.rows to go.
-   * @return Merged region.
-   * @throws Exception
-   */
-  private HRegion mergeAndVerify(final String msg, final String regionName1,
-    final String regionName2, final WAL log, final int upperbound)
-  throws Exception {
-    Merge merger = new Merge();
-    LOG.info(msg);
-    LOG.info("fs2=" + this.conf.get("fs.defaultFS"));
-    int errCode = ToolRunner.run(this.conf, merger,
-      new String[] {this.desc.getTableName().getNameAsString(), regionName1, 
regionName2}
-    );
-    assertTrue("'" + msg + "' failed with errCode " + errCode, errCode == 0);
-    HRegionInfo mergedInfo = merger.getMergedHRegionInfo();
-
-    // Now verify that we can read all the rows from regions 0, 1
-    // in the new merged region.
-    HRegion merged = HRegion.openHRegion(mergedInfo, this.desc, log, 
this.conf);
-    verifyMerge(merged, upperbound);
-    merged.close();
-    LOG.info("Verified " + msg);
-    return merged;
-  }
-
-  private void verifyMerge(final HRegion merged, final int upperbound)
-  throws IOException {
-    //Test
-    Scan scan = new Scan();
-    scan.addFamily(FAMILY);
-    InternalScanner scanner = merged.getScanner(scan);
-    try {
-    List<Cell> testRes = null;
-      while (true) {
-        testRes = new ArrayList<Cell>();
-        boolean hasNext = scanner.next(testRes);
-        if (!hasNext) {
-          break;
-        }
-      }
-    } finally {
-      scanner.close();
-    }
-
-    //!Test
-
-    for (int i = 0; i < upperbound; i++) {
-      for (int j = 0; j < rows[i].length; j++) {
-        Get get = new Get(rows[i][j]);
-        get.addFamily(FAMILY);
-        Result result = merged.get(get);
-        assertEquals(1, result.size());
-        byte [] bytes = CellUtil.cloneValue(result.rawCells()[0]);
-        assertNotNull(Bytes.toStringBinary(rows[i][j]), bytes);
-        assertTrue(Bytes.equals(bytes, rows[i][j]));
-      }
-    }
-  }
-
-  /**
-   * Test merge tool.
-   * @throws Exception
-   */
-  @Test
-  public void testMergeTool() throws Exception {
-    // First verify we can read the rows from the source regions and that they
-    // contain the right data.
-    for (int i = 0; i < regions.length; i++) {
-      for (int j = 0; j < rows[i].length; j++) {
-        Get get = new Get(rows[i][j]);
-        get.addFamily(FAMILY);
-        Result result = regions[i].get(get);
-        byte [] bytes =  CellUtil.cloneValue(result.rawCells()[0]);
-        assertNotNull(bytes);
-        assertTrue(Bytes.equals(bytes, rows[i][j]));
-      }
-      // Close the region and delete the log
-      HBaseTestingUtility.closeRegionAndWAL(regions[i]);
-    }
-    WAL log = wals.getWAL(new byte[]{}, null);
-     // Merge Region 0 and Region 1
-    HRegion merged = mergeAndVerify("merging regions 0 and 1 ",
-      this.sourceRegions[0].getRegionNameAsString(),
-      this.sourceRegions[1].getRegionNameAsString(), log, 2);
-
-    // Merge the result of merging regions 0 and 1 with region 2
-    merged = mergeAndVerify("merging regions 0+1 and 2",
-      merged.getRegionInfo().getRegionNameAsString(),
-      this.sourceRegions[2].getRegionNameAsString(), log, 3);
-
-    // Merge the result of merging regions 0, 1 and 2 with region 3
-    merged = mergeAndVerify("merging regions 0+1+2 and 3",
-      merged.getRegionInfo().getRegionNameAsString(),
-      this.sourceRegions[3].getRegionNameAsString(), log, 4);
-
-    // Merge the result of merging regions 0, 1, 2 and 3 with region 4
-    merged = mergeAndVerify("merging regions 0+1+2+3 and 4",
-      merged.getRegionInfo().getRegionNameAsString(),
-      this.sourceRegions[4].getRegionNameAsString(), log, rows.length);
-  }
-
-}
-

Reply via email to