@InterfaceAudience.Public public class HFileOutputFormat2 extends org.apache.hadoop.mapreduce.lib.output.FileOutputFormat<ImmutableBytesWritable,Cell>
HFile
s. Calling write(null,null) will forcibly roll
all HFiles being written.
Using this class as part of a MapReduce job is best done
using configureIncrementalLoad(Job, TableDescriptor, RegionLocator)
.
Modifier and Type | Class and Description |
---|---|
(package private) static class |
HFileOutputFormat2.TableInfo |
(package private) static class |
HFileOutputFormat2.WriterLength |
Modifier and Type | Field and Description |
---|---|
(package private) static String |
BLOCK_SIZE_FAMILIES_CONF_KEY |
(package private) static Function<ColumnFamilyDescriptor,String> |
blockSizeDetails
Serialize column family to block size map to configuration.
|
(package private) static String |
BLOOM_PARAM_FAMILIES_CONF_KEY |
(package private) static String |
BLOOM_TYPE_FAMILIES_CONF_KEY |
(package private) static Function<ColumnFamilyDescriptor,String> |
bloomParamDetails
Serialize column family to bloom param map to configuration.
|
(package private) static Function<ColumnFamilyDescriptor,String> |
bloomTypeDetails
Serialize column family to bloom type map to configuration.
|
(package private) static String |
COMPRESSION_FAMILIES_CONF_KEY |
static String |
COMPRESSION_OVERRIDE_CONF_KEY |
(package private) static Function<ColumnFamilyDescriptor,String> |
compressionDetails
Serialize column family to compression algorithm map to configuration.
|
(package private) static String |
DATABLOCK_ENCODING_FAMILIES_CONF_KEY |
static String |
DATABLOCK_ENCODING_OVERRIDE_CONF_KEY |
(package private) static Function<ColumnFamilyDescriptor,String> |
dataBlockEncodingDetails
Serialize column family to data block encoding map to configuration.
|
private static boolean |
DEFAULT_LOCALITY_SENSITIVE |
static String |
LOCALITY_SENSITIVE_CONF_KEY
Keep locality while generating HFiles for bulkload.
|
private static org.slf4j.Logger |
LOG |
(package private) static String |
MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY |
(package private) static String |
OUTPUT_TABLE_NAME_CONF_KEY |
static String |
STORAGE_POLICY_PROPERTY |
static String |
STORAGE_POLICY_PROPERTY_CF_PREFIX |
protected static byte[] |
tableSeparator |
Constructor and Description |
---|
HFileOutputFormat2() |
Modifier and Type | Method and Description |
---|---|
protected static byte[] |
combineTableNameSuffix(byte[] tableName,
byte[] suffix) |
(package private) static void |
configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job,
List<HFileOutputFormat2.TableInfo> multiTableInfo,
Class<? extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> cls) |
static void |
configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job,
TableDescriptor tableDescriptor,
RegionLocator regionLocator)
Configure a MapReduce Job to perform an incremental load into the given
table.
|
static void |
configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job,
Table table,
RegionLocator regionLocator)
Configure a MapReduce Job to perform an incremental load into the given
table.
|
static void |
configureIncrementalLoadMap(org.apache.hadoop.mapreduce.Job job,
TableDescriptor tableDescriptor) |
(package private) static void |
configurePartitioner(org.apache.hadoop.mapreduce.Job job,
List<ImmutableBytesWritable> splitPoints,
boolean writeMultipleTables)
Configure
job with a TotalOrderPartitioner, partitioning against
splitPoints . |
(package private) static void |
configureStoragePolicy(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
byte[] tableAndFamily,
org.apache.hadoop.fs.Path cfPath)
Configure block storage policy for CF after the directory is created.
|
(package private) static Map<byte[],Integer> |
createFamilyBlockSizeMap(org.apache.hadoop.conf.Configuration conf)
Runs inside the task to deserialize column family to block size
map from the configuration.
|
(package private) static Map<byte[],String> |
createFamilyBloomParamMap(org.apache.hadoop.conf.Configuration conf)
Runs inside the task to deserialize column family to bloom filter param
map from the configuration.
|
(package private) static Map<byte[],BloomType> |
createFamilyBloomTypeMap(org.apache.hadoop.conf.Configuration conf)
Runs inside the task to deserialize column family to bloom filter type
map from the configuration.
|
(package private) static Map<byte[],Compression.Algorithm> |
createFamilyCompressionMap(org.apache.hadoop.conf.Configuration conf)
Runs inside the task to deserialize column family to compression algorithm
map from the configuration.
|
private static Map<byte[],String> |
createFamilyConfValueMap(org.apache.hadoop.conf.Configuration conf,
String confName)
Run inside the task to deserialize column family to given conf value map.
|
(package private) static Map<byte[],DataBlockEncoding> |
createFamilyDataBlockEncodingMap(org.apache.hadoop.conf.Configuration conf)
Runs inside the task to deserialize column family to data block encoding
type map from the configuration.
|
(package private) static <V extends Cell> |
createRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext context,
org.apache.hadoop.mapreduce.OutputCommitter committer) |
org.apache.hadoop.mapreduce.RecordWriter<ImmutableBytesWritable,Cell> |
getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext context) |
private static List<ImmutableBytesWritable> |
getRegionStartKeys(List<RegionLocator> regionLocators,
boolean writeMultipleTables)
Return the start keys of all of the regions in this table,
as a list of ImmutableBytesWritable.
|
protected static byte[] |
getTableNameSuffixedWithFamily(byte[] tableName,
byte[] family) |
(package private) static String |
serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor,String> fn,
List<TableDescriptor> allTables) |
private static void |
writePartitions(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.Path partitionsPath,
List<ImmutableBytesWritable> startKeys,
boolean writeMultipleTables)
Write out a
SequenceFile that can be read by
TotalOrderPartitioner that contains the split points in startKeys. |
checkOutputSpecs, getCompressOutput, getDefaultWorkFile, getOutputCommitter, getOutputCompressorClass, getOutputName, getOutputPath, getPathForWorkFile, getUniqueFile, getWorkOutputPath, setCompressOutput, setOutputCompressorClass, setOutputName, setOutputPath
private static final org.slf4j.Logger LOG
protected static final byte[] tableSeparator
static final String COMPRESSION_FAMILIES_CONF_KEY
static final String BLOOM_TYPE_FAMILIES_CONF_KEY
static final String BLOOM_PARAM_FAMILIES_CONF_KEY
static final String BLOCK_SIZE_FAMILIES_CONF_KEY
static final String DATABLOCK_ENCODING_FAMILIES_CONF_KEY
public static final String DATABLOCK_ENCODING_OVERRIDE_CONF_KEY
public static final String COMPRESSION_OVERRIDE_CONF_KEY
public static final String LOCALITY_SENSITIVE_CONF_KEY
private static final boolean DEFAULT_LOCALITY_SENSITIVE
static final String OUTPUT_TABLE_NAME_CONF_KEY
static final String MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY
public static final String STORAGE_POLICY_PROPERTY
public static final String STORAGE_POLICY_PROPERTY_CF_PREFIX
static Function<ColumnFamilyDescriptor,String> compressionDetails
static Function<ColumnFamilyDescriptor,String> blockSizeDetails
static Function<ColumnFamilyDescriptor,String> bloomTypeDetails
static Function<ColumnFamilyDescriptor,String> bloomParamDetails
static Function<ColumnFamilyDescriptor,String> dataBlockEncodingDetails
public HFileOutputFormat2()
protected static byte[] combineTableNameSuffix(byte[] tableName, byte[] suffix)
public org.apache.hadoop.mapreduce.RecordWriter<ImmutableBytesWritable,Cell> getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext context) throws IOException, InterruptedException
getRecordWriter
in class org.apache.hadoop.mapreduce.lib.output.FileOutputFormat<ImmutableBytesWritable,Cell>
IOException
InterruptedException
protected static byte[] getTableNameSuffixedWithFamily(byte[] tableName, byte[] family)
static <V extends Cell> org.apache.hadoop.mapreduce.RecordWriter<ImmutableBytesWritable,V> createRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext context, org.apache.hadoop.mapreduce.OutputCommitter committer) throws IOException
IOException
static void configureStoragePolicy(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, byte[] tableAndFamily, org.apache.hadoop.fs.Path cfPath)
private static List<ImmutableBytesWritable> getRegionStartKeys(List<RegionLocator> regionLocators, boolean writeMultipleTables) throws IOException
IOException
private static void writePartitions(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.Path partitionsPath, List<ImmutableBytesWritable> startKeys, boolean writeMultipleTables) throws IOException
SequenceFile
that can be read by
TotalOrderPartitioner
that contains the split points in startKeys.IOException
public static void configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job, Table table, RegionLocator regionLocator) throws IOException
IOException
public static void configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job, TableDescriptor tableDescriptor, RegionLocator regionLocator) throws IOException
IOException
static void configureIncrementalLoad(org.apache.hadoop.mapreduce.Job job, List<HFileOutputFormat2.TableInfo> multiTableInfo, Class<? extends org.apache.hadoop.mapreduce.OutputFormat<?,?>> cls) throws IOException
IOException
public static void configureIncrementalLoadMap(org.apache.hadoop.mapreduce.Job job, TableDescriptor tableDescriptor) throws IOException
IOException
static Map<byte[],Compression.Algorithm> createFamilyCompressionMap(org.apache.hadoop.conf.Configuration conf)
conf
- to read the serialized values fromstatic Map<byte[],BloomType> createFamilyBloomTypeMap(org.apache.hadoop.conf.Configuration conf)
conf
- to read the serialized values fromstatic Map<byte[],String> createFamilyBloomParamMap(org.apache.hadoop.conf.Configuration conf)
conf
- to read the serialized values fromstatic Map<byte[],Integer> createFamilyBlockSizeMap(org.apache.hadoop.conf.Configuration conf)
conf
- to read the serialized values fromstatic Map<byte[],DataBlockEncoding> createFamilyDataBlockEncodingMap(org.apache.hadoop.conf.Configuration conf)
conf
- to read the serialized values fromprivate static Map<byte[],String> createFamilyConfValueMap(org.apache.hadoop.conf.Configuration conf, String confName)
conf
- to read the serialized values fromconfName
- conf key to read from the configurationstatic void configurePartitioner(org.apache.hadoop.mapreduce.Job job, List<ImmutableBytesWritable> splitPoints, boolean writeMultipleTables) throws IOException
job
with a TotalOrderPartitioner, partitioning against
splitPoints
. Cleans up the partitions file after job exists.IOException
static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor,String> fn, List<TableDescriptor> allTables) throws UnsupportedEncodingException
UnsupportedEncodingException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.