HBASE-8842 TestTokenAuthentication failing on hadoop2 build with "IllegalArgumentException: Can t get Kerberos realm"

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1499134 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-07-02 22:33:03 +00:00
parent f400db0d9b
commit ff0196f69a
1 changed files with 56 additions and 41 deletions

View File

@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.replication.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
@ -32,9 +34,9 @@ import org.apache.hadoop.hbase.client.HTable;
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.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Before;
@ -42,6 +44,8 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.ByteString;
@Category(MediumTests.class)
public class TestReplicationSink {
private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
@ -117,11 +121,12 @@ public class TestReplicationSink {
*/
@Test
public void testBatchSink() throws Exception {
HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE];
List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE);
List<Cell> cells = new ArrayList<Cell>();
for(int i = 0; i < BATCH_SIZE; i++) {
entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put);
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
Scan scan = new Scan();
ResultScanner scanRes = table1.getScanner(scan);
assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length);
@ -133,19 +138,21 @@ public class TestReplicationSink {
*/
@Test
public void testMixedPutDelete() throws Exception {
HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE/2];
List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE/2);
List<Cell> cells = new ArrayList<Cell>();
for(int i = 0; i < BATCH_SIZE/2; i++) {
entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put);
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells));
entries = new HLog.Entry[BATCH_SIZE];
entries = new ArrayList<WALEntry>(BATCH_SIZE);
cells = new ArrayList<Cell>();
for(int i = 0; i < BATCH_SIZE; i++) {
entries[i] = createEntry(TABLE_NAME1, i,
i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn);
entries.add(createEntry(TABLE_NAME1, i,
i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn, cells));
}
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
Scan scan = new Scan();
ResultScanner scanRes = table1.getScanner(scan);
assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length);
@ -157,14 +164,14 @@ public class TestReplicationSink {
*/
@Test
public void testMixedPutTables() throws Exception {
HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE];
List<WALEntry> entries = new ArrayList<WALEntry>(BATCH_SIZE/2);
List<Cell> cells = new ArrayList<Cell>();
for(int i = 0; i < BATCH_SIZE; i++) {
entries[i] =
createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1,
i, KeyValue.Type.Put);
entries.add(createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1,
i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
Scan scan = new Scan();
ResultScanner scanRes = table2.getScanner(scan);
for(Result res : scanRes) {
@ -178,18 +185,19 @@ public class TestReplicationSink {
*/
@Test
public void testMixedDeletes() throws Exception {
HLog.Entry[] entries = new HLog.Entry[3];
List<WALEntry> entries = new ArrayList<WALEntry>(3);
List<Cell> cells = new ArrayList<Cell>();
for(int i = 0; i < 3; i++) {
entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put);
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries);
entries = new HLog.Entry[3];
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
entries = new ArrayList<WALEntry>(3);
cells = new ArrayList<Cell>();
entries.add(createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn, cells));
entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
entries.add(createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn, cells));
entries[0] = createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn);
entries[1] = createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily);
entries[2] = createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn);
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
Scan scan = new Scan();
ResultScanner scanRes = table1.getScanner(scan);
@ -203,21 +211,22 @@ public class TestReplicationSink {
*/
@Test
public void testApplyDeleteBeforePut() throws Exception {
HLog.Entry[] entries = new HLog.Entry[5];
List<WALEntry> entries = new ArrayList<WALEntry>(5);
List<Cell> cells = new ArrayList<Cell>();
for(int i = 0; i < 2; i++) {
entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put);
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
entries[2] = createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily);
entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
for(int i = 3; i < 5; i++) {
entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put);
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
}
SINK.replicateEntries(entries);
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
Get get = new Get(Bytes.toBytes(1));
Result res = table1.get(get);
assertEquals(0, res.size());
}
private HLog.Entry createEntry(byte [] table, int row, KeyValue.Type type) {
private WALEntry createEntry(byte [] table, int row, KeyValue.Type type, List<Cell> cells) {
byte[] fam = Bytes.equals(table, TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2;
byte[] rowBytes = Bytes.toBytes(row);
// Just make sure we don't get the same ts for two consecutive rows with
@ -239,15 +248,21 @@ public class TestReplicationSink {
kv = new KeyValue(rowBytes, fam, null,
now, KeyValue.Type.DeleteFamily);
}
WALEntry.Builder builder = WALEntry.newBuilder();
builder.setAssociatedCellCount(1);
WALKey.Builder keyBuilder = WALKey.newBuilder();
UUID.Builder uuidBuilder = UUID.newBuilder();
uuidBuilder.setLeastSigBits(HConstants.DEFAULT_CLUSTER_ID.getLeastSignificantBits());
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
keyBuilder.setClusterId(uuidBuilder.build());
keyBuilder.setTableName(ByteString.copyFrom(table));
keyBuilder.setWriteTime(now);
keyBuilder.setEncodedRegionName(ByteString.copyFrom(HConstants.EMPTY_BYTE_ARRAY));
keyBuilder.setLogSequenceNumber(-1);
builder.setKey(keyBuilder.build());
cells.add(kv);
HLogKey key = new HLogKey(table, table, now, now,
HConstants.DEFAULT_CLUSTER_ID);
WALEdit edit = new WALEdit();
edit.add(kv);
return new HLog.Entry(key, edit);
return builder.build();
}
}