1 /**
2 *
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
10 *
11 * http://www.apache.org/licenses/LICENSE-2.0
12 *
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
18 */
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.Map;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.classification.InterfaceStability;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.HTableDescriptor;
31 import org.apache.hadoop.hbase.KeyValue;
32 import org.apache.hadoop.hbase.client.HTable;
33 import org.apache.hadoop.hbase.client.Table;
34 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
35 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
36 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
37 import org.apache.hadoop.hbase.regionserver.BloomType;
38 import org.apache.hadoop.mapreduce.Job;
39 import org.apache.hadoop.mapreduce.RecordWriter;
40 import org.apache.hadoop.mapreduce.TaskAttemptContext;
41 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
42
43 /**
44 * Writes HFiles. Passed KeyValues must arrive in order.
45 * Writes current time as the sequence id for the file. Sets the major compacted
46 * attribute on created hfiles. Calling write(null,null) will forcibly roll
47 * all HFiles being written.
48 * <p>
49 * Using this class as part of a MapReduce job is best done
50 * using {@link #configureIncrementalLoad(Job, HTable)}.
51 * @see KeyValueSortReducer
52 * @deprecated use {@link HFileOutputFormat2} instead.
53 */
54 @Deprecated
55 @InterfaceAudience.Public
56 @InterfaceStability.Stable
57 public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> {
58 private static final Log LOG = LogFactory.getLog(HFileOutputFormat.class);
59
60 // This constant is public since the client can modify this when setting
61 // up their conf object and thus refer to this symbol.
62 // It is present for backwards compatibility reasons. Use it only to
63 // override the auto-detection of datablock encoding.
64 public static final String DATABLOCK_ENCODING_OVERRIDE_CONF_KEY =
65 HFileOutputFormat2.DATABLOCK_ENCODING_OVERRIDE_CONF_KEY;
66
67 @Override
68 public RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(
69 final TaskAttemptContext context) throws IOException, InterruptedException {
70 return HFileOutputFormat2.createRecordWriter(context, this.getOutputCommitter(context));
71 }
72
73 /**
74 * Configure a MapReduce Job to perform an incremental load into the given
75 * table. This
76 * <ul>
77 * <li>Inspects the table to configure a total order partitioner</li>
78 * <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li>
79 * <li>Sets the number of reduce tasks to match the current number of regions</li>
80 * <li>Sets the output key/value class to match HFileOutputFormat's requirements</li>
81 * <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or
82 * PutSortReducer)</li>
83 * </ul>
84 * The user should be sure to set the map output value class to either KeyValue or Put before
85 * running this function.
86 */
87 public static void configureIncrementalLoad(Job job, HTable table)
88 throws IOException {
89 HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(),
90 table.getRegionLocator());
91 }
92
93 /**
94 * Runs inside the task to deserialize column family to compression algorithm
95 * map from the configuration.
96 *
97 * @param conf to read the serialized values from
98 * @return a map from column family to the configured compression algorithm
99 */
100 @InterfaceAudience.Private
101 static Map<byte[], Algorithm> createFamilyCompressionMap(Configuration
102 conf) {
103 return HFileOutputFormat2.createFamilyCompressionMap(conf);
104 }
105
106 /**
107 * Runs inside the task to deserialize column family to bloom filter type
108 * map from the configuration.
109 *
110 * @param conf to read the serialized values from
111 * @return a map from column family to the the configured bloom filter type
112 */
113 @InterfaceAudience.Private
114 static Map<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {
115 return HFileOutputFormat2.createFamilyBloomTypeMap(conf);
116 }
117
118 /**
119 * Runs inside the task to deserialize column family to block size
120 * map from the configuration.
121 *
122 * @param conf to read the serialized values from
123 * @return a map from column family to the configured block size
124 */
125 @InterfaceAudience.Private
126 static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {
127 return HFileOutputFormat2.createFamilyBlockSizeMap(conf);
128 }
129
130 /**
131 * Runs inside the task to deserialize column family to data block encoding
132 * type map from the configuration.
133 *
134 * @param conf to read the serialized values from
135 * @return a map from column family to HFileDataBlockEncoder for the
136 * configured data block type for the family
137 */
138 @InterfaceAudience.Private
139 static Map<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(
140 Configuration conf) {
141 return HFileOutputFormat2.createFamilyDataBlockEncodingMap(conf);
142 }
143
144 /**
145 * Configure <code>job</code> with a TotalOrderPartitioner, partitioning against
146 * <code>splitPoints</code>. Cleans up the partitions file after job exists.
147 */
148 static void configurePartitioner(Job job, List<ImmutableBytesWritable> splitPoints)
149 throws IOException {
150 HFileOutputFormat2.configurePartitioner(job, splitPoints);
151 }
152
153 static void configureCompression(Table table, Configuration conf) throws IOException {
154 HFileOutputFormat2.configureCompression(conf, table.getTableDescriptor());
155 }
156
157 /**
158 * Serialize column family to block size map to configuration.
159 * Invoked while configuring the MR job for incremental load.
160 *
161 * @param table to read the properties from
162 * @param conf to persist serialized values into
163 * @throws IOException
164 * on failure to read column family descriptors
165 */
166 @InterfaceAudience.Private
167 static void configureBlockSize(Table table, Configuration conf) throws IOException {
168 HFileOutputFormat2.configureBlockSize(table.getTableDescriptor(), conf);
169 }
170
171 /**
172 * Serialize column family to bloom type map to configuration.
173 * Invoked while configuring the MR job for incremental load.
174 *
175 * @param table to read the properties from
176 * @param conf to persist serialized values into
177 * @throws IOException
178 * on failure to read column family descriptors
179 */
180 @InterfaceAudience.Private
181 static void configureBloomType(Table table, Configuration conf) throws IOException {
182 HFileOutputFormat2.configureBloomType(table.getTableDescriptor(), conf);
183 }
184
185 /**
186 * Serialize column family to data block encoding map to configuration.
187 * Invoked while configuring the MR job for incremental load.
188 *
189 * @param table to read the properties from
190 * @param conf to persist serialized values into
191 * @throws IOException
192 * on failure to read column family descriptors
193 */
194 @InterfaceAudience.Private
195 static void configureDataBlockEncoding(Table table,
196 Configuration conf) throws IOException {
197 HTableDescriptor tableDescriptor = table.getTableDescriptor();
198 HFileOutputFormat2.configureDataBlockEncoding(tableDescriptor, conf);
199 }
200 }