Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,8 @@ public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken) throws IOException {
return secureBulkLoadHFiles(client, familyPaths, regionName, assignSeqNum, userToken, bulkToken,
false);
return secureBulkLoadHFiles(client, familyPaths, regionName, assignSeqNum, userToken,
bulkToken, false, null);
}

/**
Expand All @@ -132,13 +132,23 @@ public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client
* @return true if all are loaded
* @throws IOException
*/
public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken,
boolean copyFiles) throws IOException {
return secureBulkLoadHFiles(client, familyPaths, regionName, assignSeqNum, userToken,
bulkToken, false, null);
}

public boolean secureBulkLoadHFiles(final ClientService.BlockingInterface client,
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken, boolean copyFiles) throws IOException {
final Token<?> userToken, final String bulkToken,
boolean copyFiles, String clusterId) throws IOException {
BulkLoadHFileRequest request =
RequestConverter.buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum,
userToken, bulkToken, copyFiles);
userToken, bulkToken, copyFiles, clusterId);

try {
BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2577,13 +2577,23 @@ public static QuotaProtos.SpaceQuota toProtoSpaceQuota(
* name
* @return The WAL log marker for bulk loads.
*/
public static WALProtos.BulkLoadDescriptor toBulkLoadDescriptor(TableName tableName,
ByteString encodedRegionName, Map<byte[], List<Path>> storeFiles,
Map<String, Long> storeFilesSize, long bulkloadSeqId) {
return toBulkLoadDescriptor(tableName, encodedRegionName, storeFiles,
storeFilesSize, bulkloadSeqId, null);
}

public static WALProtos.BulkLoadDescriptor toBulkLoadDescriptor(TableName tableName,
ByteString encodedRegionName, Map<byte[], List<Path>> storeFiles,
Map<String, Long> storeFilesSize, long bulkloadSeqId) {
Map<String, Long> storeFilesSize, long bulkloadSeqId, String clusterId) {
BulkLoadDescriptor.Builder desc =
BulkLoadDescriptor.newBuilder()
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setEncodedRegionName(encodedRegionName).setBulkloadSeqNum(bulkloadSeqId);
if(clusterId != null) {
desc.setClusterId(clusterId);
}

for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
WALProtos.StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -562,7 +562,7 @@ public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken) {
return buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum, userToken, bulkToken,
false);
false, null);
}

/**
Expand All @@ -579,7 +579,7 @@ public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken, boolean copyFiles) {
final Token<?> userToken, final String bulkToken, boolean copyFiles, String clusterId) {
RegionSpecifier region = RequestConverter.buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);

Expand Down Expand Up @@ -617,6 +617,9 @@ public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
request.setBulkToken(bulkToken);
}
request.setCopyFile(copyFiles);
if (clusterId != null) {
request.setClusterId(clusterId);
}
return request.build();
}

Expand Down
1 change: 1 addition & 0 deletions hbase-protocol-shaded/src/main/protobuf/Client.proto
Original file line number Diff line number Diff line change
Expand Up @@ -378,6 +378,7 @@ message BulkLoadHFileRequest {
optional DelegationToken fs_token = 4;
optional string bulk_token = 5;
optional bool copy_file = 6 [default = false];
optional string cluster_id = 7;

message FamilyPath {
required bytes family = 1;
Expand Down
1 change: 1 addition & 0 deletions hbase-protocol-shaded/src/main/protobuf/WAL.proto
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@ message BulkLoadDescriptor {
required bytes encoded_region_name = 2;
repeated StoreDescriptor stores = 3;
required int64 bulkload_seq_num = 4;
optional string cluster_id = 5;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6092,7 +6092,7 @@ private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>
*/
public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
BulkLoadListener bulkLoadListener) throws IOException {
return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false);
return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false, null);
}

/**
Expand Down Expand Up @@ -6141,7 +6141,7 @@ String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile)
* @throws IOException if failed unrecoverably.
*/
public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile) throws IOException {
boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile, String clusterId) throws IOException {
long seqId = -1;
Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
Map<String, Long> storeFilesSizes = new HashMap<>();
Expand Down Expand Up @@ -6316,8 +6316,7 @@ public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> f
WALProtos.BulkLoadDescriptor loadDescriptor =
ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
storeFiles,
storeFilesSizes, seqId);
storeFiles, storeFilesSizes, seqId, clusterId);
WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
loadDescriptor, mvcc);
} catch (IOException ioe) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2364,6 +2364,14 @@ public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,
final BulkLoadHFileRequest request) throws ServiceException {
long start = EnvironmentEdgeManager.currentTime();
if(request.getClusterId() != null &&
request.getClusterId().equals(this.regionServer.clusterId)){
return BulkLoadHFileResponse.newBuilder().setLoaded(true).build();
}
String clusterId = request.getClusterId();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. Do this a few lines higher and then you can replace a few request.getClusterId() calls.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep, have addressed it on the last changes.

if(clusterId == null || clusterId.isEmpty()){
clusterId = this.regionServer.clusterId;
}
try {
checkOpen();
requestCount.increment();
Expand Down Expand Up @@ -2396,15 +2404,15 @@ public BulkLoadHFileResponse bulkLoadHFile(final RpcController controller,
}
try {
map = region.bulkLoadHFiles(familyPaths, request.getAssignSeqNum(), null,
request.getCopyFile());
request.getCopyFile(), clusterId);
} finally {
if (region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postBulkLoadHFile(familyPaths, map);
}
}
} else {
// secure bulk load
map = regionServer.secureBulkLoadManager.secureBulkLoadHFiles(region, request);
map = regionServer.secureBulkLoadManager.secureBulkLoadHFiles(region, request, clusterId);
}
BulkLoadHFileResponse.Builder builder = BulkLoadHFileResponse.newBuilder();
builder.setLoaded(map != null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,12 @@ private boolean isUserReferenced(UserGroupInformation ugi) {
}

public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
final BulkLoadHFileRequest request) throws IOException {
final BulkLoadHFileRequest request) throws IOException {
return secureBulkLoadHFiles(region, request, null);
}

public Map<byte[], List<Path>> secureBulkLoadHFiles(final HRegion region,
final BulkLoadHFileRequest request, String clusterId) throws IOException {
final List<Pair<byte[], String>> familyPaths = new ArrayList<>(request.getFamilyPathCount());
for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
familyPaths.add(new Pair<>(el.getFamily().toByteArray(), el.getPath()));
Expand Down Expand Up @@ -288,7 +293,8 @@ public Map<byte[], List<Path>> run() {
//We call bulkLoadHFiles as requesting user
//To enable access prior to staging
return region.bulkLoadHFiles(familyPaths, true,
new SecureBulkLoadListener(fs, bulkToken, conf), request.getCopyFile());
new SecureBulkLoadListener(fs, bulkToken, conf), request.getCopyFile(),
clusterId);
} catch (Exception e) {
LOG.error("Failed to complete bulk load", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,17 +87,19 @@ public class HFileReplicator {
private ThreadPoolExecutor exec;
private int maxCopyThreads;
private int copiesPerThread;
private String sourceClusterId;

public HFileReplicator(Configuration sourceClusterConf,
String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
Connection connection) throws IOException {
Connection connection, String sourceClusterId) throws IOException {
this.sourceClusterConf = sourceClusterConf;
this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
this.bulkLoadHFileMap = tableQueueMap;
this.conf = conf;
this.connection = connection;
this.sourceClusterId = sourceClusterId;

userProvider = UserProvider.instantiate(conf);
fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
Expand Down Expand Up @@ -131,6 +133,7 @@ public Void replicate() throws IOException {
LoadIncrementalHFiles loadHFiles = null;
try {
loadHFiles = new LoadIncrementalHFiles(conf);
loadHFiles.setClusterId(sourceClusterId);
} catch (Exception e) {
LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
+ " data.", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,8 +174,7 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
// invocation of this method per table and cluster id.
Map<TableName, Map<List<UUID>, List<Row>>> rowMap = new TreeMap<>();

// Map of table name Vs list of pair of family and list of hfile paths from its namespace
Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = null;
Map<String, Map<String, List<Pair<byte[], List<String>>>>> bulkLoadsPerClusters = null;

for (WALEntry entry : entries) {
TableName table =
Expand Down Expand Up @@ -204,10 +203,19 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
Cell cell = cells.current();
// Handle bulk load hfiles replication
if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
if(bulkLoadsPerClusters == null) {
bulkLoadsPerClusters = new HashMap<>();
}
// Map of table name Vs list of pair of family and list of
// hfile paths from its namespace
Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap =
bulkLoadsPerClusters.get(bld.getClusterId());
if (bulkLoadHFileMap == null) {
bulkLoadHFileMap = new HashMap<>();
bulkLoadsPerClusters.put(bld.getClusterId(), bulkLoadHFileMap);
}
buildBulkLoadHFileMap(bulkLoadHFileMap, table, cell);
buildBulkLoadHFileMap(bulkLoadHFileMap, table, bld);
} else {
// Handle wal replication
if (isNewRowOrType(previousCell, cell)) {
Expand Down Expand Up @@ -243,14 +251,20 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
LOG.debug("Finished replicating mutations.");
}

if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
LOG.debug("Started replicating bulk loaded data.");
HFileReplicator hFileReplicator =
new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
if(bulkLoadsPerClusters != null) {
for (String clusterId : bulkLoadsPerClusters.keySet()) {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

On the reported findbugs issue: We do need to iterate over the keySet, as we do need the clusterId further to forward it to HFileReplicator call.

Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap =
bulkLoadsPerClusters.get(clusterId);
if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
LOG.debug("Started replicating bulk loaded data from cluster id: {}.", clusterId);
HFileReplicator hFileReplicator =
new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,
getConnection());
hFileReplicator.replicate();
LOG.debug("Finished replicating bulk loaded data.");
getConnection(), clusterId);
hFileReplicator.replicate();
LOG.debug("Finished replicating bulk loaded data from cluster id: {}", clusterId);
}
}
}

int size = entries.size();
Expand All @@ -265,8 +279,7 @@ public void replicateEntries(List<WALEntry> entries, final CellScanner cells,

private void buildBulkLoadHFileMap(
final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, TableName table,
Cell cell) throws IOException {
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
BulkLoadDescriptor bld) throws IOException {
List<StoreDescriptor> storesList = bld.getStoresList();
int storesSize = storesList.size();
for (int j = 0; j < storesSize; j++) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {

private String bulkToken;

private String clusterId;

/**
* Represents an HFile waiting to be loaded. An queue is used in this class in order to support
* the case where a region has split during the process of the load. When this happens, the HFile
Expand Down Expand Up @@ -542,7 +544,7 @@ protected byte[] rpcCall() throws Exception {
try (Table table = conn.getTable(getTableName())) {
secureClient = new SecureBulkLoadClient(getConf(), table);
success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
assignSeqIds, fsDelegationToken.getUserToken(), bulkToken, copyFile);
assignSeqIds, fsDelegationToken.getUserToken(), bulkToken, copyFile, clusterId);
}
return success ? regionName : null;
} finally {
Expand Down Expand Up @@ -1257,6 +1259,10 @@ public void setBulkToken(String stagingDir) {
this.bulkToken = stagingDir;
}

public void setClusterId(String clusterId) {
this.clusterId = clusterId;
}

/**
* Infers region boundaries for a new table.
* <p>
Expand Down
Loading