HBASE-16585 Rewrite the delegation token tests with Parameterized pattern

This commit is contained in:
zhangduo 2017-06-30 10:58:06 +08:00
parent 4223c0c0ac
commit 21653c31d9
2 changed files with 95 additions and 54 deletions

View File

@ -15,12 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.security.token;
import com.google.protobuf.ServiceException;
import static org.junit.Assert.assertArrayEquals;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
import org.apache.hadoop.hbase.ipc.NettyRpcClient;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
@ -42,11 +41,18 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameter;
import org.junit.runners.Parameterized.Parameters;
@RunWith(Parameterized.class)
@Category({ SecurityTests.class, MediumTests.class })
public class TestDelegationTokenWithEncryption extends SecureTestCluster {
@ -55,39 +61,51 @@ public class TestDelegationTokenWithEncryption extends SecureTestCluster {
// enable rpc encryption
TEST_UTIL.getConfiguration().set("hbase.rpc.protection", "privacy");
SecureTestCluster.setUp();
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token);
}
}
private void testPutGetWithDelegationToken(Class<? extends RpcClient> rpcImplClass)
throws IOException, ServiceException {
TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
rpcImplClass.getName());
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Table table = conn.getTable(TableName.valueOf("testtable"));) {
Put p = new Put(Bytes.toBytes("row"));
p.addColumn(Bytes.toBytes("family"),
Bytes.toBytes("data"), Bytes.toBytes("testdata"));
table.put(p);
Get g = new Get(Bytes.toBytes("row"));
Result result = table.get(g);
Assert.assertArrayEquals(Bytes.toBytes("testdata"),
result.getValue(Bytes.toBytes("family"), Bytes.toBytes("data")));
@Parameters(name = "{index}: rpcClientImpl={0}")
public static Collection<Object[]> parameters() {
return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() },
new Object[] { NettyRpcClient.class.getName() });
}
@Parameter
public String rpcClientImpl;
@Rule
public TestName testName = new TestName();
@Before
public void setUpBeforeMethod() {
TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
rpcClientImpl);
}
private TableName getTestTableName() {
return TableName.valueOf(testName.getMethodName().replaceAll("[^0-9A-Za-z]", "_"));
}
@Test
public void testPutGetWithDelegationToken() throws Exception {
TableName tableName = getTestTableName();
byte[] family = Bytes.toBytes("f");
byte[] qualifier = Bytes.toBytes("q");
byte[] row = Bytes.toBytes("row");
byte[] value = Bytes.toBytes("data");
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token);
// create the table for test
Admin admin = conn.getAdmin();
HTableDescriptor tableDescriptor = new
HTableDescriptor(new HTableDescriptor(TableName.valueOf("testtable")));
tableDescriptor.addFamily(new HColumnDescriptor("family"));
HTableDescriptor tableDescriptor = new HTableDescriptor(new HTableDescriptor(tableName));
tableDescriptor.addFamily(new HColumnDescriptor(family));
admin.createTable(tableDescriptor);
testPutGetWithDelegationToken(BlockingRpcClient.class);
testPutGetWithDelegationToken(NettyRpcClient.class);
try (Table table = conn.getTable(tableName)) {
table.put(new Put(row).addColumn(family, qualifier, value));
Result result = table.get(new Get(row));
assertArrayEquals(value, result.getValue(family, qualifier));
}
}
}
}

View File

@ -17,10 +17,16 @@
*/
package org.apache.hadoop.hbase.security.token;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertThat;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
@ -30,7 +36,6 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.BlockingRpcClient;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.NettyRpcClient;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
@ -44,18 +49,45 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameter;
import org.junit.runners.Parameterized.Parameters;
import com.google.protobuf.ServiceException;
@RunWith(Parameterized.class)
@Category({ SecurityTests.class, MediumTests.class })
public class TestGenerateDelegationToken extends SecureTestCluster {
private void testTokenAuth(Class<? extends RpcClient> rpcImplClass) throws IOException,
ServiceException {
@BeforeClass
public static void setUp() throws Exception {
SecureTestCluster.setUp();
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token);
}
}
@Parameters(name = "{index}: rpcClientImpl={0}")
public static Collection<Object[]> parameters() {
return Arrays.asList(new Object[] { BlockingRpcClient.class.getName() },
new Object[] { NettyRpcClient.class.getName() });
}
@Parameter
public String rpcClientImpl;
@Before
public void setUpBeforeMethod() {
TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
rpcImplClass.getName());
rpcClientImpl);
}
@Test
public void test() throws Exception {
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Table table = conn.getTable(TableName.META_TABLE_NAME)) {
CoprocessorRpcChannel rpcChannel = table.coprocessorService(HConstants.EMPTY_START_ROW);
@ -67,20 +99,11 @@ public class TestGenerateDelegationToken extends SecureTestCluster {
try {
service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
} catch (ServiceException e) {
AccessDeniedException exc = (AccessDeniedException) ProtobufUtil.handleRemoteException(e);
assertTrue(exc.getMessage().contains(
"Token generation only allowed for Kerberos authenticated clients"));
IOException ioe = ProtobufUtil.getRemoteException(e);
assertThat(ioe, instanceOf(AccessDeniedException.class));
assertThat(ioe.getMessage(),
containsString("Token generation only allowed for Kerberos authenticated clients"));
}
}
}
@Test
public void test() throws Exception {
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token);
testTokenAuth(BlockingRpcClient.class);
testTokenAuth(NettyRpcClient.class);
}
}
}