bbeaudreault commented on code in PR #4770:
URL: https://github.com/apache/hbase/pull/4770#discussion_r1083200018
##########
hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java:
##########
@@ -340,19 +357,21 @@ public Job createSubmittableJob(String[] args) throws
IOException {
LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
// the bulk HFile case
- if (tables.length != 1) {
- throw new IOException("Exactly one table must be specified for the
bulk export option");
- }
- TableName tableName = TableName.valueOf(tables[0]);
+ List<TableName> tableNames = getTableNameList(tables);
+
job.setMapperClass(WALCellMapper.class);
job.setReducerClass(CellSortReducer.class);
Path outputDir = new Path(hfileOutPath);
FileOutputFormat.setOutputPath(job, outputDir);
job.setMapOutputValueClass(MapReduceExtendedCell.class);
- try (Connection conn = ConnectionFactory.createConnection(conf);
- Table table = conn.getTable(tableName);
- RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
- HFileOutputFormat2.configureIncrementalLoad(job,
table.getDescriptor(), regionLocator);
+ try (Connection conn = ConnectionFactory.createConnection(conf);) {
+ List<TableInfo> tableInfoList = new ArrayList<>();
+ for (TableName tableName : tableNames) {
+ Table table = conn.getTable(tableName);
+ RegionLocator regionLocator = conn.getRegionLocator(tableName);
+ tableInfoList.add(new TableInfo(table.getDescriptor(),
regionLocator));
+ }
+ MultiTableHFileOutputFormat.configureIncrementalLoad(job,
tableInfoList);
Review Comment:
Ok so I think we might want to make one more compatibility fix here:
Previously, if a user called this method with more than 1 table it would
throw an exception.
Now, those tables will be collected and passed into
MultiTableHFileOutputFormat.
That is fine I think. However, if you call this method with just 1 table (as
most users would do today since > 1 throws an error), you also will go to
MultiTableHFileOutputFormat.
I think on line 374 here we should just be defensive and do this:
```
if (tableInfoList.size() > 1) {
MultiTableHFileOutputFormat.configureIncrementalLoad(job, tableInfoList);
} else {
TableInfo tableInfo = tableInfoList.get(0);
HFileOutputFormat2.configureIncrementalLoad(job,
tableInfo.getTableDescriptor(), tableInfo.getRegionLocator());
}
```
This is important because if you look at
HFileOutputFormat2.configureIncrementalLoad there is this:
```
boolean writeMultipleTables = false;
if (MultiTableHFileOutputFormat.class.equals(cls)) {
writeMultipleTables = true;
conf.setBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, true);
}
```
So the PR as we have it here would cause all users of WALPlayer to
automatically pick up the `writeMultipleTables` behavior, even if they just use
1 table. The writeMultipleTables behavior involves a bunch of differences from
partitioner, splits, output directory, etc.
My suggestion above would ensure that you only go into the
writeMultipleTable mode if you're actually writing to multiple tables as
indicated by `tableInfoList.size() > 1`.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]