iterator() {
- throw new UnsupportedOperationException();
- }
- }
-
protected void assertResultEquals(final HRegion region, final byte [] row,
final byte [] family, final byte [] qualifier, final long timestamp,
final byte [] value)
@@ -669,5 +446,4 @@ public abstract class HBaseTestCase extends TestCase {
Bytes.toStringBinary(actual) + ">");
}
}
-
-}
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
index 3a8f0896116..a383d669903 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
@@ -29,9 +29,6 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestCase.FlushCache;
-import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon;
-import org.apache.hadoop.hbase.HBaseTestCase.Incommon;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -102,8 +99,7 @@ public class TestMultiVersions {
Table table = new HTable(UTIL.getConfiguration(), desc.getTableName());
// TODO: Remove these deprecated classes or pull them in here if this is
// only test using them.
- Incommon incommon = new HTableIncommon(table);
- TimestampTestBase.doTestDelete(incommon, new FlushCache() {
+ TimestampTestBase.doTestDelete(table, new FlushCache() {
public void flushcache() throws IOException {
UTIL.getHBaseCluster().flushcache();
}
@@ -111,7 +107,7 @@ public class TestMultiVersions {
// Perhaps drop and readd the table between tests so the former does
// not pollute this latter? Or put into separate tests.
- TimestampTestBase.doTestTimestampScanning(incommon, new FlushCache() {
+ TimestampTestBase.doTestTimestampScanning(table, new FlushCache() {
public void flushcache() throws IOException {
UTIL.getMiniHBaseCluster().flushcache();
}
@@ -141,7 +137,7 @@ public class TestMultiVersions {
desc.addFamily(hcd);
this.admin.createTable(desc);
Put put = new Put(row, timestamp1);
- put.add(contents, contents, value1);
+ put.addColumn(contents, contents, value1);
Table table = new HTable(UTIL.getConfiguration(), desc.getTableName());
table.put(put);
// Shut down and restart the HBase cluster
@@ -154,7 +150,7 @@ public class TestMultiVersions {
table = new HTable(new Configuration(UTIL.getConfiguration()), desc.getTableName());
// Overwrite previous value
put = new Put(row, timestamp2);
- put.add(contents, contents, value2);
+ put.addColumn(contents, contents, value2);
table.put(put);
// Now verify that getRow(row, column, latest) works
Get get = new Get(row);
@@ -224,7 +220,7 @@ public class TestMultiVersions {
for (int i = 0; i < locations.size(); i++) {
for (int j = 0; j < timestamp.length; j++) {
Put put = new Put(rows[i], timestamp[j]);
- put.add(HConstants.CATALOG_FAMILY, null, timestamp[j],
+ put.addColumn(HConstants.CATALOG_FAMILY, null, timestamp[j],
Bytes.toBytes(timestamp[j]));
table.put(put);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
index 40baf41aa75..0ed01d53438 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
@@ -24,6 +24,9 @@ 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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.util.Bytes;
@@ -49,7 +52,7 @@ public class TimestampTestBase extends HBaseTestCase {
* @param flusher
* @throws IOException
*/
- public static void doTestDelete(final Incommon incommon, FlushCache flusher)
+ public static void doTestDelete(final Table incommon, FlushCache flusher)
throws IOException {
// Add values at various timestamps (Values are timestampes as bytes).
put(incommon, T0);
@@ -87,8 +90,9 @@ public class TimestampTestBase extends HBaseTestCase {
put(incommon, T1);
Delete delete = new Delete(ROW);
- delete.deleteColumns(FAMILY_NAME, QUALIFIER_NAME, T2);
- incommon.delete(delete, true);
+ delete.setWriteToWAL(true);
+ delete.addColumn(FAMILY_NAME, QUALIFIER_NAME, T2);
+ incommon.delete(delete);
// Should only be current value in set. Assert this is so
assertOnlyLatest(incommon, HConstants.LATEST_TIMESTAMP);
@@ -98,7 +102,7 @@ public class TimestampTestBase extends HBaseTestCase {
assertOnlyLatest(incommon, HConstants.LATEST_TIMESTAMP);
}
- private static void assertOnlyLatest(final Incommon incommon,
+ private static void assertOnlyLatest(final Table incommon,
final long currentTime)
throws IOException {
Get get = null;
@@ -119,7 +123,7 @@ public class TimestampTestBase extends HBaseTestCase {
* @param tss
* @throws IOException
*/
- public static void assertVersions(final Incommon incommon, final long [] tss)
+ public static void assertVersions(final Table incommon, final long [] tss)
throws IOException {
// Assert that 'latest' is what we expect.
Get get = null;
@@ -169,7 +173,7 @@ public class TimestampTestBase extends HBaseTestCase {
* @param flusher
* @throws IOException
*/
- public static void doTestTimestampScanning(final Incommon incommon,
+ public static void doTestTimestampScanning(final Table incommon,
final FlushCache flusher)
throws IOException {
// Add a couple of values for three different timestamps.
@@ -195,10 +199,12 @@ public class TimestampTestBase extends HBaseTestCase {
* @return Count of items scanned.
* @throws IOException
*/
- public static int assertScanContentTimestamp(final Incommon in, final long ts)
+ public static int assertScanContentTimestamp(final Table in, final long ts)
throws IOException {
- ScannerIncommon scanner =
- in.getScanner(COLUMNS[0], null, HConstants.EMPTY_START_ROW, ts);
+ Scan scan = new Scan(HConstants.EMPTY_START_ROW);
+ scan.addFamily(COLUMNS[0]);
+ scan.setTimeRange(0, ts);
+ ResultScanner scanner = in.getScanner(scan);
int count = 0;
try {
// TODO FIX
@@ -221,12 +227,12 @@ public class TimestampTestBase extends HBaseTestCase {
return count;
}
- public static void put(final Incommon loader, final long ts)
+ public static void put(final Table loader, final long ts)
throws IOException {
put(loader, Bytes.toBytes(ts), ts);
}
- public static void put(final Incommon loader)
+ public static void put(final Table loader)
throws IOException {
long ts = HConstants.LATEST_TIMESTAMP;
put(loader, Bytes.toBytes(ts), ts);
@@ -239,39 +245,40 @@ public class TimestampTestBase extends HBaseTestCase {
* @param ts
* @throws IOException
*/
- public static void put(final Incommon loader, final byte [] bytes,
+ public static void put(final Table loader, final byte [] bytes,
final long ts)
throws IOException {
Put put = new Put(ROW, ts);
put.setDurability(Durability.SKIP_WAL);
- put.add(FAMILY_NAME, QUALIFIER_NAME, bytes);
+ put.addColumn(FAMILY_NAME, QUALIFIER_NAME, bytes);
loader.put(put);
}
- public static void delete(final Incommon loader) throws IOException {
+ public static void delete(final Table loader) throws IOException {
delete(loader, null);
}
- public static void delete(final Incommon loader, final byte [] column)
+ public static void delete(final Table loader, final byte [] column)
throws IOException {
delete(loader, column, HConstants.LATEST_TIMESTAMP);
}
- public static void delete(final Incommon loader, final long ts)
+ public static void delete(final Table loader, final long ts)
throws IOException {
delete(loader, null, ts);
}
- public static void delete(final Incommon loader, final byte [] column,
+ public static void delete(final Table loader, final byte [] column,
final long ts)
throws IOException {
Delete delete = ts == HConstants.LATEST_TIMESTAMP?
new Delete(ROW): new Delete(ROW, ts);
- delete.deleteColumn(FAMILY_NAME, QUALIFIER_NAME, ts);
- loader.delete(delete, true);
+ delete.addColumn(FAMILY_NAME, QUALIFIER_NAME, ts);
+ delete.setWriteToWAL(true);
+ loader.delete(delete);
}
- public static Result get(final Incommon loader) throws IOException {
+ public static Result get(final Table loader) throws IOException {
return loader.get(new Get(ROW));
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionTable.java
new file mode 100644
index 00000000000..29e93d6097a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionTable.java
@@ -0,0 +1,329 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
+import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
+/**
+ * An implementation of {@link Table} that sits directly on a Region; it decorates the passed in
+ * Region instance with the Table API. Some API is not implemented yet (throws
+ * {@link UnsupportedOperationException}).
+ *
+ * Use as an instance of a {@link Table} in-the-small -- no networking or servers
+ * necessary -- or to write a test that can run directly against the datastore and then
+ * over the network.
+ */
+public class RegionTable implements Table {
+ private final Region region;
+
+ /**
+ * @param region Region to decorate with Table API.
+ */
+ public RegionTable(final Region region) {
+ this.region = region;
+ }
+
+ @Override
+ public TableName getName() {
+ return this.region.getTableDesc().getTableName();
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public HTableDescriptor getTableDescriptor() throws IOException {
+ return this.region.getTableDesc();
+ }
+
+ @Override
+ public boolean exists(Get get) throws IOException {
+ if (!get.isCheckExistenceOnly()) throw new IllegalArgumentException();
+ return get(get) != null;
+ }
+
+ @Override
+ public boolean[] existsAll(List gets) throws IOException {
+ boolean [] results = new boolean[gets.size()];
+ int index = 0;
+ for (Get get: gets) {
+ results[index++] = exists(get);
+ }
+ return results;
+ }
+
+ @Override
+ public void batch(List extends Row> actions, Object[] results)
+ throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Object[] batch(List extends Row> actions) throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void batchCallback(List extends Row> actions, Object[] results,
+ Callback callback)
+ throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Object[] batchCallback(List extends Row> actions, Callback callback)
+ throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Result get(Get get) throws IOException {
+ return this.region.get(get);
+ }
+
+ @Override
+ public Result[] get(List gets) throws IOException {
+ Result [] results = new Result[gets.size()];
+ int index = 0;
+ for (Get get: gets) {
+ results[index++] = get(get);
+ }
+ return results;
+ }
+
+ static class RegionScannerToResultScannerAdaptor implements ResultScanner {
+ private static final Result [] EMPTY_RESULT_ARRAY = new Result[0];
+ private final RegionScanner regionScanner;
+
+ RegionScannerToResultScannerAdaptor(final RegionScanner regionScanner) {
+ this.regionScanner = regionScanner;
+ }
+
+ @Override
+ public Iterator iterator() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Result next() throws IOException {
+ List cells = new ArrayList();
+ return regionScanner.next(cells)? Result.create(cells): null;
+ }
+
+ @Override
+ public Result[] next(int nbRows) throws IOException {
+ List results = new ArrayList(nbRows);
+ for (int i = 0; i < nbRows; i++) {
+ Result result = next();
+ if (result == null) break;
+ results.add(result);
+ }
+ return results.toArray(EMPTY_RESULT_ARRAY);
+ }
+
+ @Override
+ public void close() {
+ try {
+ regionScanner.close();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ };
+
+ @Override
+ public ResultScanner getScanner(Scan scan) throws IOException {
+ return new RegionScannerToResultScannerAdaptor(this.region.getScanner(scan));
+ }
+
+ @Override
+ public ResultScanner getScanner(byte[] family) throws IOException {
+ return getScanner(new Scan().addFamily(family));
+ }
+
+ @Override
+ public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
+ return getScanner(new Scan().addColumn(family, qualifier));
+ }
+
+ @Override
+ public void put(Put put) throws IOException {
+ this.region.put(put);
+ }
+
+ @Override
+ public void put(List puts) throws IOException {
+ for (Put put: puts) put(put);
+ }
+
+ @Override
+ public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp,
+ byte[] value, Put put)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void delete(Delete delete) throws IOException {
+ this.region.delete(delete);
+ }
+
+ @Override
+ public void delete(List deletes) throws IOException {
+ for(Delete delete: deletes) delete(delete);
+ }
+
+ @Override
+ public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value,
+ Delete delete)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+ CompareOp compareOp, byte[] value, Delete delete)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void mutateRow(RowMutations rm) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Result append(Append append) throws IOException {
+ return this.region.append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
+ }
+
+ @Override
+ public Result increment(Increment increment) throws IOException {
+ return this.region.increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
+ }
+
+ @Override
+ public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount,
+ Durability durability)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * This call will NOT close the underlying region.
+ */
+ @Override
+ public void close() throws IOException {
+ }
+
+ @Override
+ public CoprocessorRpcChannel coprocessorService(byte[] row) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Map coprocessorService(Class service, byte[] startKey,
+ byte[] endKey, Call callable)
+ throws ServiceException, Throwable {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void coprocessorService(Class service, byte[] startKey,
+ byte[] endKey, Call callable, Callback callback)
+ throws ServiceException, Throwable {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long getWriteBufferSize() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void setWriteBufferSize(long writeBufferSize) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Map batchCoprocessorService(MethodDescriptor
+ methodDescriptor, Message request,
+ byte[] startKey, byte[] endKey, R responsePrototype)
+ throws ServiceException, Throwable {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void batchCoprocessorService(MethodDescriptor methodDescriptor,
+ Message request, byte[] startKey, byte[] endKey, R responsePrototype, Callback callback)
+ throws ServiceException, Throwable {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOp compareOp,
+ byte[] value, RowMutations mutation)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index fc2d66ecc53..3170e042e58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -37,8 +37,6 @@ import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
@@ -47,25 +45,25 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
-import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
-import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL;
import org.junit.After;
import org.junit.Assume;
import org.junit.Before;
@@ -84,7 +82,6 @@ import org.mockito.stubbing.Answer;
@Category(MediumTests.class)
public class TestCompaction {
@Rule public TestName name = new TestName();
- private static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
protected Configuration conf = UTIL.getConfiguration();
@@ -146,15 +143,15 @@ public class TestCompaction {
int jmax = (int) Math.ceil(15.0/compactionThreshold);
byte [] pad = new byte[1000]; // 1 KB chunk
for (int i = 0; i < compactionThreshold; i++) {
- HRegionIncommon loader = new HRegionIncommon(r);
+ Table loader = new RegionTable(r);
Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
p.setDurability(Durability.SKIP_WAL);
for (int j = 0; j < jmax; j++) {
- p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
+ p.addColumn(COLUMN_FAMILY, Bytes.toBytes(j), pad);
}
HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
loader.put(p);
- loader.flushcache();
+ r.flush(true);
}
HRegion spyR = spy(r);
@@ -228,9 +225,9 @@ public class TestCompaction {
}
private void createStoreFile(final HRegion region, String family) throws IOException {
- HRegionIncommon loader = new HRegionIncommon(region);
+ Table loader = new RegionTable(region);
HBaseTestCase.addContent(loader, family);
- loader.flushcache();
+ region.flush(true);
}
@Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
index 48a805a26ab..97e1d6d61d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
@@ -48,6 +47,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
@@ -197,7 +197,7 @@ public class TestMajorCompaction {
createStoreFile(r);
}
// Add more content.
- HBaseTestCase.addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
+ HBaseTestCase.addContent(new RegionTable(r), Bytes.toString(COLUMN_FAMILY));
// Now there are about 5 versions of each column.
// Default is that there only 3 (MAXVERSIONS) versions allowed per column.
@@ -386,13 +386,13 @@ public class TestMajorCompaction {
}
private void createStoreFile(final Region region, String family) throws IOException {
- HRegionIncommon loader = new HRegionIncommon(region);
+ Table loader = new RegionTable(region);
HBaseTestCase.addContent(loader, family);
loader.flushcache();
}
private void createSmallerStoreFile(final Region region) throws IOException {
- HRegionIncommon loader = new HRegionIncommon(region);
+ Table loader = new RegionTable(region);
HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
"bbb").getBytes(), null);
loader.flushcache();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java
index 3a1db686a84..396d8e62b70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinorCompaction.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
@@ -161,7 +161,7 @@ public class TestMinorCompaction {
}
private void testMinorCompactionWithDelete(Delete delete, int expectedResultsAfterDelete) throws Exception {
- HRegionIncommon loader = new HRegionIncommon(r);
+ Table loader = new RegionTable(r);
for (int i = 0; i < compactionThreshold + 1; i++) {
HBaseTestCase.addContent(loader, Bytes.toString(fam1), Bytes.toString(col1), firstRowBytes,
thirdRowBytes, i);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
index 2f250fd2616..14e19e392af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
@@ -36,8 +36,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
-import org.apache.hadoop.hbase.HBaseTestCase.ScannerIncommon;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@@ -49,7 +47,9 @@ 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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
@@ -99,11 +99,10 @@ public class TestScanner {
private static final long START_CODE = Long.MAX_VALUE;
- private HRegion r;
- private HRegionIncommon region;
+ private HRegion region;
private byte[] firstRowBytes, secondRowBytes, thirdRowBytes;
- final private byte[] col1, col2;
+ final private byte[] col1;
public TestScanner() {
super();
@@ -115,7 +114,6 @@ public class TestScanner {
thirdRowBytes = START_KEY_BYTES.clone();
thirdRowBytes[START_KEY_BYTES.length - 1] += 2;
col1 = Bytes.toBytes("column1");
- col2 = Bytes.toBytes("column2");
}
/**
@@ -127,14 +125,14 @@ public class TestScanner {
byte [] startrow = Bytes.toBytes("bbb");
byte [] stoprow = Bytes.toBytes("ccc");
try {
- this.r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- HBaseTestCase.addContent(this.r, HConstants.CATALOG_FAMILY);
+ this.region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ HBaseTestCase.addContent(this.region, HConstants.CATALOG_FAMILY);
List results = new ArrayList();
// Do simple test of getting one row only first.
Scan scan = new Scan(Bytes.toBytes("abc"), Bytes.toBytes("abd"));
scan.addFamily(HConstants.CATALOG_FAMILY);
- InternalScanner s = r.getScanner(scan);
+ InternalScanner s = region.getScanner(scan);
int count = 0;
while (s.next(results)) {
count++;
@@ -145,7 +143,7 @@ public class TestScanner {
scan = new Scan(startrow, stoprow);
scan.addFamily(HConstants.CATALOG_FAMILY);
- s = r.getScanner(scan);
+ s = region.getScanner(scan);
count = 0;
Cell kv = null;
results = new ArrayList();
@@ -162,14 +160,14 @@ public class TestScanner {
assertTrue(count > 10);
s.close();
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
void rowPrefixFilter(Scan scan) throws IOException {
List results = new ArrayList();
scan.addFamily(HConstants.CATALOG_FAMILY);
- InternalScanner s = r.getScanner(scan);
+ InternalScanner s = region.getScanner(scan);
boolean hasMore = true;
while (hasMore) {
hasMore = s.next(results);
@@ -185,7 +183,7 @@ public class TestScanner {
void rowInclusiveStopFilter(Scan scan, byte[] stopRow) throws IOException {
List results = new ArrayList();
scan.addFamily(HConstants.CATALOG_FAMILY);
- InternalScanner s = r.getScanner(scan);
+ InternalScanner s = region.getScanner(scan);
boolean hasMore = true;
while (hasMore) {
hasMore = s.next(results);
@@ -200,8 +198,8 @@ public class TestScanner {
@Test
public void testFilters() throws IOException {
try {
- this.r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- HBaseTestCase.addContent(this.r, HConstants.CATALOG_FAMILY);
+ this.region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ HBaseTestCase.addContent(this.region, HConstants.CATALOG_FAMILY);
byte [] prefix = Bytes.toBytes("ab");
Filter newFilter = new PrefixFilter(prefix);
Scan scan = new Scan();
@@ -215,7 +213,7 @@ public class TestScanner {
rowInclusiveStopFilter(scan, stopRow);
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
@@ -227,10 +225,10 @@ public class TestScanner {
@Test
public void testRaceBetweenClientAndTimeout() throws Exception {
try {
- this.r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- HBaseTestCase.addContent(this.r, HConstants.CATALOG_FAMILY);
+ this.region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ HBaseTestCase.addContent(this.region, HConstants.CATALOG_FAMILY);
Scan scan = new Scan();
- InternalScanner s = r.getScanner(scan);
+ InternalScanner s = region.getScanner(scan);
List results = new ArrayList();
try {
s.next(results);
@@ -242,7 +240,7 @@ public class TestScanner {
return;
}
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
@@ -252,71 +250,70 @@ public class TestScanner {
@Test
public void testScanner() throws IOException {
try {
- r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- region = new HRegionIncommon(r);
+ region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ Table table = new RegionTable(region);
// Write information to the meta table
Put put = new Put(ROW_KEY, System.currentTimeMillis());
- put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+ put.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
REGION_INFO.toByteArray());
- region.put(put);
+ table.put(put);
// What we just committed is in the memstore. Verify that we can get
// it back both with scanning and get
scan(false, null);
- getRegionInfo();
+ getRegionInfo(table);
// Close and re-open
- ((HRegion)r).close();
- r = HRegion.openHRegion(r, null);
- region = new HRegionIncommon(r);
+ ((HRegion)region).close();
+ region = HRegion.openHRegion(region, null);
+ table = new RegionTable(region);
// Verify we can get the data back now that it is on disk.
scan(false, null);
- getRegionInfo();
+ getRegionInfo(table);
// Store some new information
String address = HConstants.LOCALHOST_IP + ":" + HBaseTestingUtility.randomFreePort();
put = new Put(ROW_KEY, System.currentTimeMillis());
- put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
+ put.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(address));
// put.add(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE));
- region.put(put);
+ table.put(put);
// Validate that we can still get the HRegionInfo, even though it is in
// an older row on disk and there is a newer row in the memstore
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
// flush cache
-
- region.flushcache();
+ this.region.flush(true);
// Validate again
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
// Close and reopen
- ((HRegion)r).close();
- r = HRegion.openHRegion(r,null);
- region = new HRegionIncommon(r);
+ ((HRegion)region).close();
+ region = HRegion.openHRegion(region,null);
+ table = new RegionTable(region);
// Validate again
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
// Now update the information again
@@ -324,38 +321,36 @@ public class TestScanner {
put = new Put(ROW_KEY, System.currentTimeMillis());
- put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
- Bytes.toBytes(address));
- region.put(put);
+ put.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes.toBytes(address));
+ table.put(put);
// Validate again
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
// flush cache
-
- region.flushcache();
+ region.flush(true);
// Validate again
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
// Close and reopen
- ((HRegion)r).close();
- r = HRegion.openHRegion(r,null);
- region = new HRegionIncommon(r);
+ ((HRegion)region).close();
+ region = HRegion.openHRegion(region,null);
+ table = new RegionTable(region);
// Validate again
scan(true, address.toString());
- getRegionInfo();
+ getRegionInfo(table);
} finally {
// clean up
- HRegion.closeHRegion(r);
+ HRegion.closeHRegion(region);
}
}
@@ -387,7 +382,7 @@ public class TestScanner {
for (int ii = 0; ii < EXPLICIT_COLS.length; ii++) {
scan.addColumn(COLS[0], EXPLICIT_COLS[ii]);
}
- scanner = r.getScanner(scan);
+ scanner = region.getScanner(scan);
while (scanner.next(results)) {
assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
@@ -448,10 +443,10 @@ public class TestScanner {
/** Use get to retrieve the HRegionInfo and validate it */
- private void getRegionInfo() throws IOException {
+ private void getRegionInfo(Table table) throws IOException {
Get get = new Get(ROW_KEY);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
- Result result = region.get(get);
+ Result result = table.get(get);
byte [] bytes = result.value();
validateRegionInfo(bytes);
}
@@ -464,10 +459,11 @@ public class TestScanner {
*/
@Test
public void testScanAndSyncFlush() throws Exception {
- this.r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- HRegionIncommon hri = new HRegionIncommon(r);
+ this.region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ Table hri = new RegionTable(region);
try {
- LOG.info("Added: " + HBaseTestCase.addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
+ LOG.info("Added: " +
+ HBaseTestCase.addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
Bytes.toString(HConstants.REGIONINFO_QUALIFIER)));
int count = count(hri, -1, false);
assertEquals(count, count(hri, 100, false)); // do a sync flush.
@@ -475,7 +471,7 @@ public class TestScanner {
LOG.error("Failed", e);
throw e;
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
@@ -487,10 +483,11 @@ public class TestScanner {
*/
@Test
public void testScanAndRealConcurrentFlush() throws Exception {
- this.r = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
- HRegionIncommon hri = new HRegionIncommon(r);
+ this.region = TEST_UTIL.createLocalHRegion(TESTTABLEDESC, null, null);
+ Table hri = new RegionTable(region);
try {
- LOG.info("Added: " + HBaseTestCase.addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
+ LOG.info("Added: " +
+ HBaseTestCase.addContent(hri, Bytes.toString(HConstants.CATALOG_FAMILY),
Bytes.toString(HConstants.REGIONINFO_QUALIFIER)));
int count = count(hri, -1, false);
assertEquals(count, count(hri, 100, true)); // do a true concurrent background thread flush
@@ -498,7 +495,7 @@ public class TestScanner {
LOG.error("Failed", e);
throw e;
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
@@ -512,8 +509,8 @@ public class TestScanner {
@SuppressWarnings("deprecation")
public void testScanAndConcurrentMajorCompact() throws Exception {
HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name.getMethodName());
- this.r = TEST_UTIL.createLocalHRegion(htd, null, null);
- HRegionIncommon hri = new HRegionIncommon(r);
+ this.region = TEST_UTIL.createLocalHRegion(htd, null, null);
+ Table hri = new RegionTable(region);
try {
HBaseTestCase.addContent(hri, Bytes.toString(fam1), Bytes.toString(col1),
@@ -524,18 +521,18 @@ public class TestScanner {
Delete dc = new Delete(firstRowBytes);
/* delete column1 of firstRow */
dc.deleteColumns(fam1, col1);
- r.delete(dc);
- r.flush(true);
+ region.delete(dc);
+ region.flush(true);
HBaseTestCase.addContent(hri, Bytes.toString(fam1), Bytes.toString(col1),
secondRowBytes, thirdRowBytes);
HBaseTestCase.addContent(hri, Bytes.toString(fam2), Bytes.toString(col1),
secondRowBytes, thirdRowBytes);
- r.flush(true);
+ region.flush(true);
- InternalScanner s = r.getScanner(new Scan());
+ InternalScanner s = region.getScanner(new Scan());
// run a major compact, column1 of firstRow will be cleaned.
- r.compact(true);
+ region.compact(true);
List results = new ArrayList();
s.next(results);
@@ -555,7 +552,7 @@ public class TestScanner {
assertTrue(CellUtil.matchingFamily(results.get(0), fam1));
assertTrue(CellUtil.matchingFamily(results.get(1), fam2));
} finally {
- HRegion.closeHRegion(this.r);
+ HRegion.closeHRegion(this.region);
}
}
@@ -567,19 +564,20 @@ public class TestScanner {
* @return Count of rows found.
* @throws IOException
*/
- private int count(final HRegionIncommon hri, final int flushIndex,
- boolean concurrent)
+ private int count(final Table countTable, final int flushIndex, boolean concurrent)
throws IOException {
LOG.info("Taking out counting scan");
- ScannerIncommon s = hri.getScanner(HConstants.CATALOG_FAMILY, EXPLICIT_COLS,
- HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP);
- List values = new ArrayList();
+ Scan scan = new Scan();
+ for (byte [] qualifier: EXPLICIT_COLS) {
+ scan.addColumn(HConstants.CATALOG_FAMILY, qualifier);
+ }
+ ResultScanner s = countTable.getScanner(scan);
int count = 0;
boolean justFlushed = false;
- while (s.next(values)) {
+ while (s.next() != null) {
if (justFlushed) {
LOG.info("after next() just after next flush");
- justFlushed=false;
+ justFlushed = false;
}
count++;
if (flushIndex == count) {
@@ -587,7 +585,7 @@ public class TestScanner {
Thread t = new Thread() {
public void run() {
try {
- hri.flushcache();
+ region.flush(true);
LOG.info("Finishing flush");
} catch (IOException e) {
LOG.info("Failed flush cache");
@@ -607,5 +605,4 @@ public class TestScanner {
LOG.info("Found " + count + " items");
return count;
}
-
-}
+}
\ No newline at end of file
| | | | | | | | | | | | | | |