Index: hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java (revision 1512509) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java (working copy) @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.ipc.RequestContext; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService; @@ -151,8 +152,7 @@ try { getAccessController().prePrepareBulkLoad(env); String bulkToken = createStagingDir(baseStagingDir, - getActiveUser(), - TableName.valueOf(request.getTableName().toByteArray())).toString(); + getActiveUser(), ProtobufUtil.toTableName(request.getTableName())).toString(); done.run(PrepareBulkLoadResponse.newBuilder().setBulkToken(bulkToken).build()); } catch (IOException e) { ResponseConverter.setControllerException(controller, e);