Skip to content

Commit 8426f4d

Browse files
committed
Only use ExtendedCellSerialization for WALPlayer
1 parent bf8ca51 commit 8426f4d

3 files changed

Lines changed: 43 additions & 4 deletions

File tree

hbase-backup/pom.xml

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@
2929
<artifactId>hbase-backup</artifactId>
3030
<name>Apache HBase - Backup</name>
3131
<description>Backup for HBase</description>
32-
<!-- trigger build -->
3332
<dependencies>
3433
<!-- Intra-project dependencies -->
3534
<dependency>

hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java

Lines changed: 39 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import java.nio.charset.Charset;
3131
import java.util.ArrayList;
3232
import java.util.Arrays;
33+
import java.util.Collections;
3334
import java.util.List;
3435
import java.util.Map;
3536
import java.util.Map.Entry;
@@ -159,6 +160,15 @@ protected static byte[] combineTableNameSuffix(byte[] tableName, byte[] suffix)
159160
static final String MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY =
160161
"hbase.mapreduce.use.multi.table.hfileoutputformat";
161162

163+
/**
164+
* ExtendedCell and ExtendedCellSerialization are InterfaceAudience.Private. We expose this config
165+
* package-private for internal usage for jobs like WALPlayer which need to use features of
166+
* ExtendedCell.
167+
*/
168+
static final String EXTENDED_CELL_SERIALIZATION_ENABLED_KEY =
169+
"hbase.mapreduce.hfileoutputformat.extendedcell.enabled";
170+
static final boolean EXTENDED_CELL_SERIALIZATION_ENABLED_DEFULT = false;
171+
162172
public static final String REMOTE_CLUSTER_CONF_PREFIX = "hbase.hfileoutputformat.remote.cluster.";
163173
public static final String REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY =
164174
REMOTE_CLUSTER_CONF_PREFIX + "zookeeper.quorum";
@@ -619,13 +629,12 @@ static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo,
619629
LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
620630
}
621631

622-
// Order matters here. Hadoop's SerializationFactory runs through serializations in the order
623-
// they are registered. Register ExtendedCellSerialization before CellSerialization because both
624-
// work for ExtendedCells but ExtendedCellSerialization handles them properly.
625632
conf.setStrings("io.serializations", conf.get("io.serializations"),
626633
MutationSerialization.class.getName(), ResultSerialization.class.getName(),
627634
ExtendedCellSerialization.class.getName(), CellSerialization.class.getName());
628635

636+
mergeSerializations(conf);
637+
629638
if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
630639
LOG.info("bulkload locality sensitive enabled");
631640
}
@@ -673,6 +682,33 @@ static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo,
673682
LOG.info("Incremental output configured for tables: " + StringUtils.join(allTableNames, ","));
674683
}
675684

685+
private static void mergeSerializations(Configuration conf) {
686+
List<String> serializations = new ArrayList<>();
687+
688+
// add any existing values that have been set
689+
String[] existing = conf.getStrings("io.serializations");
690+
if (existing != null) {
691+
Collections.addAll(serializations, existing);
692+
}
693+
694+
serializations.add(MutationSerialization.class.getName());
695+
serializations.add(ResultSerialization.class.getName());
696+
697+
// Add ExtendedCellSerialization, if configured. Order matters here. Hadoop's
698+
// SerializationFactory runs through serializations in the order they are registered.
699+
// We want to register ExtendedCellSerialization before CellSerialization because both
700+
// work for ExtendedCells but only ExtendedCellSerialization handles them properly.
701+
if (
702+
conf.getBoolean(EXTENDED_CELL_SERIALIZATION_ENABLED_KEY,
703+
EXTENDED_CELL_SERIALIZATION_ENABLED_DEFULT)
704+
) {
705+
serializations.add(ExtendedCellSerialization.class.getName());
706+
}
707+
serializations.add(CellSerialization.class.getName());
708+
709+
conf.setStrings("io.serializations", serializations.toArray(new String[0]));
710+
}
711+
676712
public static void configureIncrementalLoadMap(Job job, TableDescriptor tableDescriptor)
677713
throws IOException {
678714
Configuration conf = job.getConfiguration();

hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -316,6 +316,10 @@ public Job createSubmittableJob(String[] args) throws IOException {
316316
if (hfileOutPath != null) {
317317
LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
318318

319+
// WALPlayer needs ExtendedCellSerialization so that sequenceId can be propagated when
320+
// sorting cells in CellSortReducer
321+
conf.setBoolean(HFileOutputFormat2.EXTENDED_CELL_SERIALIZATION_ENABLED_KEY, true);
322+
319323
// the bulk HFile case
320324
List<TableName> tableNames = getTableNameList(tables);
321325

0 commit comments

Comments
 (0)