001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.mapreduce;
019
020import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;
021import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
022import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;
023import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
024
025import java.io.IOException;
026import java.io.UnsupportedEncodingException;
027import java.net.InetSocketAddress;
028import java.net.URLDecoder;
029import java.net.URLEncoder;
030import java.util.ArrayList;
031import java.util.Arrays;
032import java.util.List;
033import java.util.Map;
034import java.util.Map.Entry;
035import java.util.Set;
036import java.util.TreeMap;
037import java.util.TreeSet;
038import java.util.UUID;
039import java.util.function.Function;
040import java.util.stream.Collectors;
041import org.apache.commons.lang3.StringUtils;
042import org.apache.hadoop.conf.Configuration;
043import org.apache.hadoop.fs.FileSystem;
044import org.apache.hadoop.fs.Path;
045import org.apache.hadoop.hbase.Cell;
046import org.apache.hadoop.hbase.CellUtil;
047import org.apache.hadoop.hbase.HConstants;
048import org.apache.hadoop.hbase.HRegionLocation;
049import org.apache.hadoop.hbase.HTableDescriptor;
050import org.apache.hadoop.hbase.KeyValue;
051import org.apache.hadoop.hbase.PrivateCellUtil;
052import org.apache.hadoop.hbase.TableName;
053import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
054import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
055import org.apache.hadoop.hbase.client.Connection;
056import org.apache.hadoop.hbase.client.ConnectionFactory;
057import org.apache.hadoop.hbase.client.Put;
058import org.apache.hadoop.hbase.client.RegionLocator;
059import org.apache.hadoop.hbase.client.Table;
060import org.apache.hadoop.hbase.client.TableDescriptor;
061import org.apache.hadoop.hbase.fs.HFileSystem;
062import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
063import org.apache.hadoop.hbase.io.compress.Compression;
064import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
065import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
066import org.apache.hadoop.hbase.io.hfile.CacheConfig;
067import org.apache.hadoop.hbase.io.hfile.HFile;
068import org.apache.hadoop.hbase.io.hfile.HFileContext;
069import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
070import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
071import org.apache.hadoop.hbase.regionserver.BloomType;
072import org.apache.hadoop.hbase.regionserver.HStore;
073import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
074import org.apache.hadoop.hbase.regionserver.StoreUtils;
075import org.apache.hadoop.hbase.util.BloomFilterUtil;
076import org.apache.hadoop.hbase.util.Bytes;
077import org.apache.hadoop.hbase.util.CommonFSUtils;
078import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
079import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
080import org.apache.hadoop.io.NullWritable;
081import org.apache.hadoop.io.SequenceFile;
082import org.apache.hadoop.io.Text;
083import org.apache.hadoop.mapreduce.Job;
084import org.apache.hadoop.mapreduce.OutputCommitter;
085import org.apache.hadoop.mapreduce.OutputFormat;
086import org.apache.hadoop.mapreduce.RecordWriter;
087import org.apache.hadoop.mapreduce.TaskAttemptContext;
088import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
089import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
090import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
091import org.apache.yetus.audience.InterfaceAudience;
092import org.slf4j.Logger;
093import org.slf4j.LoggerFactory;
094
095/**
096 * Writes HFiles. Passed Cells must arrive in order. Writes current time as the sequence id for the
097 * file. Sets the major compacted attribute on created {@link HFile}s. Calling write(null,null) will
098 * forcibly roll all HFiles being written.
099 * <p>
100 * Using this class as part of a MapReduce job is best done using
101 * {@link #configureIncrementalLoad(Job, TableDescriptor, RegionLocator)}.
102 */
103@InterfaceAudience.Public
104public class HFileOutputFormat2 extends FileOutputFormat<ImmutableBytesWritable, Cell> {
105  private static final Logger LOG = LoggerFactory.getLogger(HFileOutputFormat2.class);
106
107  static class TableInfo {
108    private TableDescriptor tableDesctiptor;
109    private RegionLocator regionLocator;
110
111    public TableInfo(TableDescriptor tableDesctiptor, RegionLocator regionLocator) {
112      this.tableDesctiptor = tableDesctiptor;
113      this.regionLocator = regionLocator;
114    }
115
116    /**
117     * The modification for the returned HTD doesn't affect the inner TD.
118     * @return A clone of inner table descriptor
119     * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getTableDescriptor()}
120     *             instead.
121     * @see #getTableDescriptor()
122     * @see <a href="https://issues.apache.org/jira/browse/HBASE-18241">HBASE-18241</a>
123     */
124    @Deprecated
125    public HTableDescriptor getHTableDescriptor() {
126      return new HTableDescriptor(tableDesctiptor);
127    }
128
129    public TableDescriptor getTableDescriptor() {
130      return tableDesctiptor;
131    }
132
133    public RegionLocator getRegionLocator() {
134      return regionLocator;
135    }
136  }
137
138  protected static final byte[] tableSeparator = Bytes.toBytes(";");
139
140  protected static byte[] combineTableNameSuffix(byte[] tableName, byte[] suffix) {
141    return Bytes.add(tableName, tableSeparator, suffix);
142  }
143
144  // The following constants are private since these are used by
145  // HFileOutputFormat2 to internally transfer data between job setup and
146  // reducer run using conf.
147  // These should not be changed by the client.
148  static final String COMPRESSION_FAMILIES_CONF_KEY =
149    "hbase.hfileoutputformat.families.compression";
150  static final String BLOOM_TYPE_FAMILIES_CONF_KEY = "hbase.hfileoutputformat.families.bloomtype";
151  static final String BLOOM_PARAM_FAMILIES_CONF_KEY = "hbase.hfileoutputformat.families.bloomparam";
152  static final String BLOCK_SIZE_FAMILIES_CONF_KEY = "hbase.mapreduce.hfileoutputformat.blocksize";
153  static final String DATABLOCK_ENCODING_FAMILIES_CONF_KEY =
154    "hbase.mapreduce.hfileoutputformat.families.datablock.encoding";
155
156  // This constant is public since the client can modify this when setting
157  // up their conf object and thus refer to this symbol.
158  // It is present for backwards compatibility reasons. Use it only to
159  // override the auto-detection of datablock encoding and compression.
160  public static final String DATABLOCK_ENCODING_OVERRIDE_CONF_KEY =
161    "hbase.mapreduce.hfileoutputformat.datablock.encoding";
162  public static final String COMPRESSION_OVERRIDE_CONF_KEY =
163    "hbase.mapreduce.hfileoutputformat.compression";
164
165  /**
166   * Keep locality while generating HFiles for bulkload. See HBASE-12596
167   */
168  public static final String LOCALITY_SENSITIVE_CONF_KEY =
169    "hbase.bulkload.locality.sensitive.enabled";
170  private static final boolean DEFAULT_LOCALITY_SENSITIVE = true;
171  static final String OUTPUT_TABLE_NAME_CONF_KEY = "hbase.mapreduce.hfileoutputformat.table.name";
172  static final String MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY =
173    "hbase.mapreduce.use.multi.table.hfileoutputformat";
174
175  public static final String REMOTE_CLUSTER_CONF_PREFIX = "hbase.hfileoutputformat.remote.cluster.";
176  public static final String REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY =
177    REMOTE_CLUSTER_CONF_PREFIX + "zookeeper.quorum";
178  public static final String REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY =
179    REMOTE_CLUSTER_CONF_PREFIX + "zookeeper." + HConstants.CLIENT_PORT_STR;
180  public static final String REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY =
181    REMOTE_CLUSTER_CONF_PREFIX + HConstants.ZOOKEEPER_ZNODE_PARENT;
182
183  public static final String STORAGE_POLICY_PROPERTY = HStore.BLOCK_STORAGE_POLICY_KEY;
184  public static final String STORAGE_POLICY_PROPERTY_CF_PREFIX = STORAGE_POLICY_PROPERTY + ".";
185
186  @Override
187  public RecordWriter<ImmutableBytesWritable, Cell>
188    getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException {
189    return createRecordWriter(context, this.getOutputCommitter(context));
190  }
191
192  protected static byte[] getTableNameSuffixedWithFamily(byte[] tableName, byte[] family) {
193    return combineTableNameSuffix(tableName, family);
194  }
195
196  static <V extends Cell> RecordWriter<ImmutableBytesWritable, V> createRecordWriter(
197    final TaskAttemptContext context, final OutputCommitter committer) throws IOException {
198
199    // Get the path of the temporary output file
200    final Path outputDir = ((FileOutputCommitter) committer).getWorkPath();
201    final Configuration conf = context.getConfiguration();
202    final boolean writeMultipleTables =
203      conf.getBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, false);
204    final String writeTableNames = conf.get(OUTPUT_TABLE_NAME_CONF_KEY);
205    if (writeTableNames == null || writeTableNames.isEmpty()) {
206      throw new IllegalArgumentException("" + OUTPUT_TABLE_NAME_CONF_KEY + " cannot be empty");
207    }
208    final FileSystem fs = outputDir.getFileSystem(conf);
209    // These configs. are from hbase-*.xml
210    final long maxsize =
211      conf.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
212    // Invented config. Add to hbase-*.xml if other than default compression.
213    final String defaultCompressionStr =
214      conf.get("hfile.compression", Compression.Algorithm.NONE.getName());
215    final Algorithm defaultCompression = HFileWriterImpl.compressionByName(defaultCompressionStr);
216    String compressionStr = conf.get(COMPRESSION_OVERRIDE_CONF_KEY);
217    final Algorithm overriddenCompression =
218      compressionStr != null ? Compression.getCompressionAlgorithmByName(compressionStr) : null;
219    final boolean compactionExclude =
220      conf.getBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", false);
221    final Set<String> allTableNames = Arrays
222      .stream(writeTableNames.split(Bytes.toString(tableSeparator))).collect(Collectors.toSet());
223
224    // create a map from column family to the compression algorithm
225    final Map<byte[], Algorithm> compressionMap = createFamilyCompressionMap(conf);
226    final Map<byte[], BloomType> bloomTypeMap = createFamilyBloomTypeMap(conf);
227    final Map<byte[], String> bloomParamMap = createFamilyBloomParamMap(conf);
228    final Map<byte[], Integer> blockSizeMap = createFamilyBlockSizeMap(conf);
229
230    String dataBlockEncodingStr = conf.get(DATABLOCK_ENCODING_OVERRIDE_CONF_KEY);
231    final Map<byte[], DataBlockEncoding> datablockEncodingMap =
232      createFamilyDataBlockEncodingMap(conf);
233    final DataBlockEncoding overriddenEncoding =
234      dataBlockEncodingStr != null ? DataBlockEncoding.valueOf(dataBlockEncodingStr) : null;
235
236    return new RecordWriter<ImmutableBytesWritable, V>() {
237      // Map of families to writers and how much has been output on the writer.
238      private final Map<byte[], WriterLength> writers = new TreeMap<>(Bytes.BYTES_COMPARATOR);
239      private final Map<byte[], byte[]> previousRows = new TreeMap<>(Bytes.BYTES_COMPARATOR);
240      private final long now = EnvironmentEdgeManager.currentTime();
241
242      @Override
243      public void write(ImmutableBytesWritable row, V cell) throws IOException {
244        Cell kv = cell;
245        // null input == user explicitly wants to flush
246        if (row == null && kv == null) {
247          rollWriters(null);
248          return;
249        }
250
251        byte[] rowKey = CellUtil.cloneRow(kv);
252        int length = (PrivateCellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT;
253        byte[] family = CellUtil.cloneFamily(kv);
254        byte[] tableNameBytes = null;
255        if (writeMultipleTables) {
256          tableNameBytes = MultiTableHFileOutputFormat.getTableName(row.get());
257          tableNameBytes = TableName.valueOf(tableNameBytes).toBytes();
258          if (!allTableNames.contains(Bytes.toString(tableNameBytes))) {
259            throw new IllegalArgumentException(
260              "TableName " + Bytes.toString(tableNameBytes) + " not expected");
261          }
262        } else {
263          tableNameBytes = Bytes.toBytes(writeTableNames);
264        }
265        Path tableRelPath = getTableRelativePath(tableNameBytes);
266        byte[] tableAndFamily = getTableNameSuffixedWithFamily(tableNameBytes, family);
267        WriterLength wl = this.writers.get(tableAndFamily);
268
269        // If this is a new column family, verify that the directory exists
270        if (wl == null) {
271          Path writerPath = null;
272          if (writeMultipleTables) {
273            writerPath = new Path(outputDir, new Path(tableRelPath, Bytes.toString(family)));
274          } else {
275            writerPath = new Path(outputDir, Bytes.toString(family));
276          }
277          fs.mkdirs(writerPath);
278          configureStoragePolicy(conf, fs, tableAndFamily, writerPath);
279        }
280
281        // This can only happen once a row is finished though
282        if (
283          wl != null && wl.written + length >= maxsize
284            && Bytes.compareTo(this.previousRows.get(family), rowKey) != 0
285        ) {
286          rollWriters(wl);
287        }
288
289        // create a new WAL writer, if necessary
290        if (wl == null || wl.writer == null) {
291          if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
292            HRegionLocation loc = null;
293
294            String tableName = Bytes.toString(tableNameBytes);
295            if (tableName != null) {
296              try (
297                Connection connection =
298                  ConnectionFactory.createConnection(createRemoteClusterConf(conf));
299                RegionLocator locator = connection.getRegionLocator(TableName.valueOf(tableName))) {
300                loc = locator.getRegionLocation(rowKey);
301              } catch (Throwable e) {
302                LOG.warn("Something wrong locating rowkey {} in {}", Bytes.toString(rowKey),
303                  tableName, e);
304                loc = null;
305              }
306            }
307
308            if (null == loc) {
309              LOG.trace("Failed get of location, use default writer {}", Bytes.toString(rowKey));
310              wl = getNewWriter(tableNameBytes, family, conf, null);
311            } else {
312              LOG.debug("First rowkey: [{}]", Bytes.toString(rowKey));
313              InetSocketAddress initialIsa =
314                new InetSocketAddress(loc.getHostname(), loc.getPort());
315              if (initialIsa.isUnresolved()) {
316                LOG.trace("Failed resolve address {}, use default writer", loc.getHostnamePort());
317                wl = getNewWriter(tableNameBytes, family, conf, null);
318              } else {
319                LOG.debug("Use favored nodes writer: {}", initialIsa.getHostString());
320                wl = getNewWriter(tableNameBytes, family, conf,
321                  new InetSocketAddress[] { initialIsa });
322              }
323            }
324          } else {
325            wl = getNewWriter(tableNameBytes, family, conf, null);
326          }
327        }
328
329        // we now have the proper WAL writer. full steam ahead
330        PrivateCellUtil.updateLatestStamp(cell, this.now);
331        wl.writer.append(kv);
332        wl.written += length;
333
334        // Copy the row so we know when a row transition.
335        this.previousRows.put(family, rowKey);
336      }
337
338      private Path getTableRelativePath(byte[] tableNameBytes) {
339        String tableName = Bytes.toString(tableNameBytes);
340        String[] tableNameParts = tableName.split(":");
341        Path tableRelPath = new Path(tableName.split(":")[0]);
342        if (tableNameParts.length > 1) {
343          tableRelPath = new Path(tableRelPath, tableName.split(":")[1]);
344        }
345        return tableRelPath;
346      }
347
348      private void rollWriters(WriterLength writerLength) throws IOException {
349        if (writerLength != null) {
350          closeWriter(writerLength);
351        } else {
352          for (WriterLength wl : this.writers.values()) {
353            closeWriter(wl);
354          }
355        }
356      }
357
358      private void closeWriter(WriterLength wl) throws IOException {
359        if (wl.writer != null) {
360          LOG.info(
361            "Writer=" + wl.writer.getPath() + ((wl.written == 0) ? "" : ", wrote=" + wl.written));
362          close(wl.writer);
363          wl.writer = null;
364        }
365        wl.written = 0;
366      }
367
368      private Configuration createRemoteClusterConf(Configuration conf) {
369        final Configuration newConf = new Configuration(conf);
370
371        final String quorum = conf.get(REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY);
372        final String clientPort = conf.get(REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY);
373        final String parent = conf.get(REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY);
374
375        if (quorum != null && clientPort != null && parent != null) {
376          newConf.set(HConstants.ZOOKEEPER_QUORUM, quorum);
377          newConf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.parseInt(clientPort));
378          newConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parent);
379        }
380
381        for (Entry<String, String> entry : conf) {
382          String key = entry.getKey();
383          if (
384            REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY.equals(key)
385              || REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY.equals(key)
386              || REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY.equals(key)
387          ) {
388            // Handled them above
389            continue;
390          }
391
392          if (entry.getKey().startsWith(REMOTE_CLUSTER_CONF_PREFIX)) {
393            String originalKey = entry.getKey().substring(REMOTE_CLUSTER_CONF_PREFIX.length());
394            if (!originalKey.isEmpty()) {
395              newConf.set(originalKey, entry.getValue());
396            }
397          }
398        }
399
400        return newConf;
401      }
402
403      /*
404       * Create a new StoreFile.Writer.
405       * @return A WriterLength, containing a new StoreFile.Writer.
406       */
407      @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "BX_UNBOXING_IMMEDIATELY_REBOXED",
408          justification = "Not important")
409      private WriterLength getNewWriter(byte[] tableName, byte[] family, Configuration conf,
410        InetSocketAddress[] favoredNodes) throws IOException {
411        byte[] tableAndFamily = getTableNameSuffixedWithFamily(tableName, family);
412        Path familydir = new Path(outputDir, Bytes.toString(family));
413        if (writeMultipleTables) {
414          familydir =
415            new Path(outputDir, new Path(getTableRelativePath(tableName), Bytes.toString(family)));
416        }
417        WriterLength wl = new WriterLength();
418        Algorithm compression = overriddenCompression;
419        compression = compression == null ? compressionMap.get(tableAndFamily) : compression;
420        compression = compression == null ? defaultCompression : compression;
421        BloomType bloomType = bloomTypeMap.get(tableAndFamily);
422        bloomType = bloomType == null ? BloomType.NONE : bloomType;
423        String bloomParam = bloomParamMap.get(tableAndFamily);
424        if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
425          conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, bloomParam);
426        }
427        Integer blockSize = blockSizeMap.get(tableAndFamily);
428        blockSize = blockSize == null ? HConstants.DEFAULT_BLOCKSIZE : blockSize;
429        DataBlockEncoding encoding = overriddenEncoding;
430        encoding = encoding == null ? datablockEncodingMap.get(tableAndFamily) : encoding;
431        encoding = encoding == null ? DataBlockEncoding.NONE : encoding;
432        HFileContextBuilder contextBuilder = new HFileContextBuilder().withCompression(compression)
433          .withDataBlockEncoding(encoding).withChecksumType(StoreUtils.getChecksumType(conf))
434          .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blockSize)
435          .withColumnFamily(family).withTableName(tableName);
436
437        if (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
438          contextBuilder.withIncludesTags(true);
439        }
440
441        HFileContext hFileContext = contextBuilder.build();
442        if (null == favoredNodes) {
443          wl.writer =
444            new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, fs).withOutputDir(familydir)
445              .withBloomType(bloomType).withFileContext(hFileContext).build();
446        } else {
447          wl.writer = new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, new HFileSystem(fs))
448            .withOutputDir(familydir).withBloomType(bloomType).withFileContext(hFileContext)
449            .withFavoredNodes(favoredNodes).build();
450        }
451
452        this.writers.put(tableAndFamily, wl);
453        return wl;
454      }
455
456      private void close(final StoreFileWriter w) throws IOException {
457        if (w != null) {
458          w.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(EnvironmentEdgeManager.currentTime()));
459          w.appendFileInfo(BULKLOAD_TASK_KEY, Bytes.toBytes(context.getTaskAttemptID().toString()));
460          w.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(true));
461          w.appendFileInfo(EXCLUDE_FROM_MINOR_COMPACTION_KEY, Bytes.toBytes(compactionExclude));
462          w.appendTrackedTimestampsToMetadata();
463          w.close();
464        }
465      }
466
467      @Override
468      public void close(TaskAttemptContext c) throws IOException, InterruptedException {
469        for (WriterLength wl : this.writers.values()) {
470          close(wl.writer);
471        }
472      }
473    };
474  }
475
476  /**
477   * Configure block storage policy for CF after the directory is created.
478   */
479  static void configureStoragePolicy(final Configuration conf, final FileSystem fs,
480    byte[] tableAndFamily, Path cfPath) {
481    if (null == conf || null == fs || null == tableAndFamily || null == cfPath) {
482      return;
483    }
484
485    String policy = conf.get(STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(tableAndFamily),
486      conf.get(STORAGE_POLICY_PROPERTY));
487    CommonFSUtils.setStoragePolicy(fs, cfPath, policy);
488  }
489
490  /*
491   * Data structure to hold a Writer and amount of data written on it.
492   */
493  static class WriterLength {
494    long written = 0;
495    StoreFileWriter writer = null;
496  }
497
498  /**
499   * Return the start keys of all of the regions in this table, as a list of ImmutableBytesWritable.
500   */
501  private static List<ImmutableBytesWritable> getRegionStartKeys(List<RegionLocator> regionLocators,
502    boolean writeMultipleTables) throws IOException {
503
504    ArrayList<ImmutableBytesWritable> ret = new ArrayList<>();
505    for (RegionLocator regionLocator : regionLocators) {
506      TableName tableName = regionLocator.getName();
507      LOG.info("Looking up current regions for table " + tableName);
508      byte[][] byteKeys = regionLocator.getStartKeys();
509      for (byte[] byteKey : byteKeys) {
510        byte[] fullKey = byteKey; // HFileOutputFormat2 use case
511        if (writeMultipleTables) {
512          // MultiTableHFileOutputFormat use case
513          fullKey = combineTableNameSuffix(tableName.getName(), byteKey);
514        }
515        if (LOG.isDebugEnabled()) {
516          LOG.debug("SplitPoint startkey for " + tableName + ": " + Bytes.toStringBinary(fullKey));
517        }
518        ret.add(new ImmutableBytesWritable(fullKey));
519      }
520    }
521    return ret;
522  }
523
524  /**
525   * Write out a {@link SequenceFile} that can be read by {@link TotalOrderPartitioner} that
526   * contains the split points in startKeys.
527   */
528  @SuppressWarnings("deprecation")
529  private static void writePartitions(Configuration conf, Path partitionsPath,
530    List<ImmutableBytesWritable> startKeys, boolean writeMultipleTables) throws IOException {
531    LOG.info("Writing partition information to " + partitionsPath);
532    if (startKeys.isEmpty()) {
533      throw new IllegalArgumentException("No regions passed");
534    }
535
536    // We're generating a list of split points, and we don't ever
537    // have keys < the first region (which has an empty start key)
538    // so we need to remove it. Otherwise we would end up with an
539    // empty reducer with index 0
540    TreeSet<ImmutableBytesWritable> sorted = new TreeSet<>(startKeys);
541    ImmutableBytesWritable first = sorted.first();
542    if (writeMultipleTables) {
543      first =
544        new ImmutableBytesWritable(MultiTableHFileOutputFormat.getSuffix(sorted.first().get()));
545    }
546    if (!first.equals(HConstants.EMPTY_BYTE_ARRAY)) {
547      throw new IllegalArgumentException(
548        "First region of table should have empty start key. Instead has: "
549          + Bytes.toStringBinary(first.get()));
550    }
551    sorted.remove(sorted.first());
552
553    // Write the actual file
554    FileSystem fs = partitionsPath.getFileSystem(conf);
555    SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, partitionsPath,
556      ImmutableBytesWritable.class, NullWritable.class);
557
558    try {
559      for (ImmutableBytesWritable startKey : sorted) {
560        writer.append(startKey, NullWritable.get());
561      }
562    } finally {
563      writer.close();
564    }
565  }
566
567  /**
568   * Configure a MapReduce Job to perform an incremental load into the given table. This
569   * <ul>
570   * <li>Inspects the table to configure a total order partitioner</li>
571   * <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li>
572   * <li>Sets the number of reduce tasks to match the current number of regions</li>
573   * <li>Sets the output key/value class to match HFileOutputFormat2's requirements</li>
574   * <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or
575   * PutSortReducer)</li>
576   * <li>Sets the HBase cluster key to load region locations for locality-sensitive</li>
577   * </ul>
578   * The user should be sure to set the map output value class to either KeyValue or Put before
579   * running this function.
580   */
581  public static void configureIncrementalLoad(Job job, Table table, RegionLocator regionLocator)
582    throws IOException {
583    configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
584    configureRemoteCluster(job, table.getConfiguration());
585  }
586
587  /**
588   * Configure a MapReduce Job to perform an incremental load into the given table. This
589   * <ul>
590   * <li>Inspects the table to configure a total order partitioner</li>
591   * <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li>
592   * <li>Sets the number of reduce tasks to match the current number of regions</li>
593   * <li>Sets the output key/value class to match HFileOutputFormat2's requirements</li>
594   * <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or
595   * PutSortReducer)</li>
596   * </ul>
597   * The user should be sure to set the map output value class to either KeyValue or Put before
598   * running this function.
599   */
600  public static void configureIncrementalLoad(Job job, TableDescriptor tableDescriptor,
601    RegionLocator regionLocator) throws IOException {
602    ArrayList<TableInfo> singleTableInfo = new ArrayList<>();
603    singleTableInfo.add(new TableInfo(tableDescriptor, regionLocator));
604    configureIncrementalLoad(job, singleTableInfo, HFileOutputFormat2.class);
605  }
606
607  static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo,
608    Class<? extends OutputFormat<?, ?>> cls) throws IOException {
609    Configuration conf = job.getConfiguration();
610    job.setOutputKeyClass(ImmutableBytesWritable.class);
611    job.setOutputValueClass(MapReduceExtendedCell.class);
612    job.setOutputFormatClass(cls);
613
614    if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
615      throw new IllegalArgumentException("Duplicate entries found in TableInfo argument");
616    }
617    boolean writeMultipleTables = false;
618    if (MultiTableHFileOutputFormat.class.equals(cls)) {
619      writeMultipleTables = true;
620      conf.setBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, true);
621    }
622    // Based on the configured map output class, set the correct reducer to properly
623    // sort the incoming values.
624    // TODO it would be nice to pick one or the other of these formats.
625    if (
626      KeyValue.class.equals(job.getMapOutputValueClass())
627        || MapReduceExtendedCell.class.equals(job.getMapOutputValueClass())
628    ) {
629      job.setReducerClass(CellSortReducer.class);
630    } else if (Put.class.equals(job.getMapOutputValueClass())) {
631      job.setReducerClass(PutSortReducer.class);
632    } else if (Text.class.equals(job.getMapOutputValueClass())) {
633      job.setReducerClass(TextSortReducer.class);
634    } else {
635      LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
636    }
637
638    conf.setStrings("io.serializations", conf.get("io.serializations"),
639      MutationSerialization.class.getName(), ResultSerialization.class.getName(),
640      CellSerialization.class.getName());
641
642    if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
643      LOG.info("bulkload locality sensitive enabled");
644    }
645
646    /* Now get the region start keys for every table required */
647    List<String> allTableNames = new ArrayList<>(multiTableInfo.size());
648    List<RegionLocator> regionLocators = new ArrayList<>(multiTableInfo.size());
649    List<TableDescriptor> tableDescriptors = new ArrayList<>(multiTableInfo.size());
650
651    for (TableInfo tableInfo : multiTableInfo) {
652      regionLocators.add(tableInfo.getRegionLocator());
653      allTableNames.add(tableInfo.getRegionLocator().getName().getNameAsString());
654      tableDescriptors.add(tableInfo.getTableDescriptor());
655    }
656    // Record tablenames for creating writer by favored nodes, and decoding compression,
657    // block size and other attributes of columnfamily per table
658    conf.set(OUTPUT_TABLE_NAME_CONF_KEY,
659      StringUtils.join(allTableNames, Bytes.toString(tableSeparator)));
660    List<ImmutableBytesWritable> startKeys =
661      getRegionStartKeys(regionLocators, writeMultipleTables);
662    // Use table's region boundaries for TOP split points.
663    LOG.info("Configuring " + startKeys.size() + " reduce partitions "
664      + "to match current region count for all tables");
665    job.setNumReduceTasks(startKeys.size());
666
667    configurePartitioner(job, startKeys, writeMultipleTables);
668    // Set compression algorithms based on column families
669
670    conf.set(COMPRESSION_FAMILIES_CONF_KEY,
671      serializeColumnFamilyAttribute(compressionDetails, tableDescriptors));
672    conf.set(BLOCK_SIZE_FAMILIES_CONF_KEY,
673      serializeColumnFamilyAttribute(blockSizeDetails, tableDescriptors));
674    conf.set(BLOOM_TYPE_FAMILIES_CONF_KEY,
675      serializeColumnFamilyAttribute(bloomTypeDetails, tableDescriptors));
676    conf.set(BLOOM_PARAM_FAMILIES_CONF_KEY,
677      serializeColumnFamilyAttribute(bloomParamDetails, tableDescriptors));
678    conf.set(DATABLOCK_ENCODING_FAMILIES_CONF_KEY,
679      serializeColumnFamilyAttribute(dataBlockEncodingDetails, tableDescriptors));
680
681    TableMapReduceUtil.addDependencyJars(job);
682    TableMapReduceUtil.initCredentials(job);
683    LOG.info("Incremental output configured for tables: " + StringUtils.join(allTableNames, ","));
684  }
685
686  public static void configureIncrementalLoadMap(Job job, TableDescriptor tableDescriptor)
687    throws IOException {
688    Configuration conf = job.getConfiguration();
689
690    job.setOutputKeyClass(ImmutableBytesWritable.class);
691    job.setOutputValueClass(MapReduceExtendedCell.class);
692    job.setOutputFormatClass(HFileOutputFormat2.class);
693
694    ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);
695    singleTableDescriptor.add(tableDescriptor);
696
697    conf.set(OUTPUT_TABLE_NAME_CONF_KEY, tableDescriptor.getTableName().getNameAsString());
698    // Set compression algorithms based on column families
699    conf.set(COMPRESSION_FAMILIES_CONF_KEY,
700      serializeColumnFamilyAttribute(compressionDetails, singleTableDescriptor));
701    conf.set(BLOCK_SIZE_FAMILIES_CONF_KEY,
702      serializeColumnFamilyAttribute(blockSizeDetails, singleTableDescriptor));
703    conf.set(BLOOM_TYPE_FAMILIES_CONF_KEY,
704      serializeColumnFamilyAttribute(bloomTypeDetails, singleTableDescriptor));
705    conf.set(BLOOM_PARAM_FAMILIES_CONF_KEY,
706      serializeColumnFamilyAttribute(bloomParamDetails, singleTableDescriptor));
707    conf.set(DATABLOCK_ENCODING_FAMILIES_CONF_KEY,
708      serializeColumnFamilyAttribute(dataBlockEncodingDetails, singleTableDescriptor));
709
710    TableMapReduceUtil.addDependencyJars(job);
711    TableMapReduceUtil.initCredentials(job);
712    LOG.info("Incremental table " + tableDescriptor.getTableName() + " output configured.");
713  }
714
715  /**
716   * Configure HBase cluster key for remote cluster to load region location for locality-sensitive
717   * if it's enabled. It's not necessary to call this method explicitly when the cluster key for
718   * HBase cluster to be used to load region location is configured in the job configuration. Call
719   * this method when another HBase cluster key is configured in the job configuration. For example,
720   * you should call when you load data from HBase cluster A using {@link TableInputFormat} and
721   * generate hfiles for HBase cluster B. Otherwise, HFileOutputFormat2 fetch location from cluster
722   * A and locality-sensitive won't working correctly.
723   * {@link #configureIncrementalLoad(Job, Table, RegionLocator)} calls this method using
724   * {@link Table#getConfiguration} as clusterConf. See HBASE-25608.
725   * @param job         which has configuration to be updated
726   * @param clusterConf which contains cluster key of the HBase cluster to be locality-sensitive
727   * @see #configureIncrementalLoad(Job, Table, RegionLocator)
728   * @see #LOCALITY_SENSITIVE_CONF_KEY
729   * @see #REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY
730   * @see #REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY
731   * @see #REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY
732   */
733  public static void configureRemoteCluster(Job job, Configuration clusterConf) {
734    Configuration conf = job.getConfiguration();
735
736    if (!conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {
737      return;
738    }
739
740    final String quorum = clusterConf.get(HConstants.ZOOKEEPER_QUORUM);
741    final int clientPort = clusterConf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT,
742      HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT);
743    final String parent =
744      clusterConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
745
746    conf.set(REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY, quorum);
747    conf.setInt(REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY, clientPort);
748    conf.set(REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY, parent);
749
750    LOG.info("ZK configs for remote cluster of bulkload is configured: " + quorum + ":" + clientPort
751      + "/" + parent);
752  }
753
754  /**
755   * Runs inside the task to deserialize column family to compression algorithm map from the
756   * configuration.
757   * @param conf to read the serialized values from
758   * @return a map from column family to the configured compression algorithm
759   */
760  @InterfaceAudience.Private
761  static Map<byte[], Algorithm> createFamilyCompressionMap(Configuration conf) {
762    Map<byte[], String> stringMap = createFamilyConfValueMap(conf, COMPRESSION_FAMILIES_CONF_KEY);
763    Map<byte[], Algorithm> compressionMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
764    for (Map.Entry<byte[], String> e : stringMap.entrySet()) {
765      Algorithm algorithm = HFileWriterImpl.compressionByName(e.getValue());
766      compressionMap.put(e.getKey(), algorithm);
767    }
768    return compressionMap;
769  }
770
771  /**
772   * Runs inside the task to deserialize column family to bloom filter type map from the
773   * configuration.
774   * @param conf to read the serialized values from
775   * @return a map from column family to the the configured bloom filter type
776   */
777  @InterfaceAudience.Private
778  static Map<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {
779    Map<byte[], String> stringMap = createFamilyConfValueMap(conf, BLOOM_TYPE_FAMILIES_CONF_KEY);
780    Map<byte[], BloomType> bloomTypeMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
781    for (Map.Entry<byte[], String> e : stringMap.entrySet()) {
782      BloomType bloomType = BloomType.valueOf(e.getValue());
783      bloomTypeMap.put(e.getKey(), bloomType);
784    }
785    return bloomTypeMap;
786  }
787
788  /**
789   * Runs inside the task to deserialize column family to bloom filter param map from the
790   * configuration.
791   * @param conf to read the serialized values from
792   * @return a map from column family to the the configured bloom filter param
793   */
794  @InterfaceAudience.Private
795  static Map<byte[], String> createFamilyBloomParamMap(Configuration conf) {
796    return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY);
797  }
798
799  /**
800   * Runs inside the task to deserialize column family to block size map from the configuration.
801   * @param conf to read the serialized values from
802   * @return a map from column family to the configured block size
803   */
804  @InterfaceAudience.Private
805  static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {
806    Map<byte[], String> stringMap = createFamilyConfValueMap(conf, BLOCK_SIZE_FAMILIES_CONF_KEY);
807    Map<byte[], Integer> blockSizeMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
808    for (Map.Entry<byte[], String> e : stringMap.entrySet()) {
809      Integer blockSize = Integer.parseInt(e.getValue());
810      blockSizeMap.put(e.getKey(), blockSize);
811    }
812    return blockSizeMap;
813  }
814
815  /**
816   * Runs inside the task to deserialize column family to data block encoding type map from the
817   * configuration.
818   * @param conf to read the serialized values from
819   * @return a map from column family to HFileDataBlockEncoder for the configured data block type
820   *         for the family
821   */
822  @InterfaceAudience.Private
823  static Map<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(Configuration conf) {
824    Map<byte[], String> stringMap =
825      createFamilyConfValueMap(conf, DATABLOCK_ENCODING_FAMILIES_CONF_KEY);
826    Map<byte[], DataBlockEncoding> encoderMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
827    for (Map.Entry<byte[], String> e : stringMap.entrySet()) {
828      encoderMap.put(e.getKey(), DataBlockEncoding.valueOf((e.getValue())));
829    }
830    return encoderMap;
831  }
832
833  /**
834   * Run inside the task to deserialize column family to given conf value map.
835   * @param conf     to read the serialized values from
836   * @param confName conf key to read from the configuration
837   * @return a map of column family to the given configuration value
838   */
839  private static Map<byte[], String> createFamilyConfValueMap(Configuration conf, String confName) {
840    Map<byte[], String> confValMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
841    String confVal = conf.get(confName, "");
842    for (String familyConf : confVal.split("&")) {
843      String[] familySplit = familyConf.split("=");
844      if (familySplit.length != 2) {
845        continue;
846      }
847      try {
848        confValMap.put(Bytes.toBytes(URLDecoder.decode(familySplit[0], "UTF-8")),
849          URLDecoder.decode(familySplit[1], "UTF-8"));
850      } catch (UnsupportedEncodingException e) {
851        // will not happen with UTF-8 encoding
852        throw new AssertionError(e);
853      }
854    }
855    return confValMap;
856  }
857
858  /**
859   * Configure <code>job</code> with a TotalOrderPartitioner, partitioning against
860   * <code>splitPoints</code>. Cleans up the partitions file after job exists.
861   */
862  static void configurePartitioner(Job job, List<ImmutableBytesWritable> splitPoints,
863    boolean writeMultipleTables) throws IOException {
864    Configuration conf = job.getConfiguration();
865    // create the partitions file
866    FileSystem fs = FileSystem.get(conf);
867    String hbaseTmpFsDir =
868      conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
869    Path partitionsPath = new Path(hbaseTmpFsDir, "partitions_" + UUID.randomUUID());
870    fs.makeQualified(partitionsPath);
871    writePartitions(conf, partitionsPath, splitPoints, writeMultipleTables);
872    fs.deleteOnExit(partitionsPath);
873
874    // configure job to use it
875    job.setPartitionerClass(TotalOrderPartitioner.class);
876    TotalOrderPartitioner.setPartitionFile(conf, partitionsPath);
877  }
878
879  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
880      value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
881  @InterfaceAudience.Private
882  static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, String> fn,
883    List<TableDescriptor> allTables) throws UnsupportedEncodingException {
884    StringBuilder attributeValue = new StringBuilder();
885    int i = 0;
886    for (TableDescriptor tableDescriptor : allTables) {
887      if (tableDescriptor == null) {
888        // could happen with mock table instance
889        // CODEREVIEW: Can I set an empty string in conf if mock table instance?
890        return "";
891      }
892      for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) {
893        if (i++ > 0) {
894          attributeValue.append('&');
895        }
896        attributeValue.append(URLEncoder
897          .encode(Bytes.toString(combineTableNameSuffix(tableDescriptor.getTableName().getName(),
898            familyDescriptor.getName())), "UTF-8"));
899        attributeValue.append('=');
900        attributeValue.append(URLEncoder.encode(fn.apply(familyDescriptor), "UTF-8"));
901      }
902    }
903    // Get rid of the last ampersand
904    return attributeValue.toString();
905  }
906
907  /**
908   * Serialize column family to compression algorithm map to configuration. Invoked while
909   * configuring the MR job for incremental load.
910   */
911  @InterfaceAudience.Private
912  static Function<ColumnFamilyDescriptor, String> compressionDetails =
913    familyDescriptor -> familyDescriptor.getCompressionType().getName();
914
915  /**
916   * Serialize column family to block size map to configuration. Invoked while configuring the MR
917   * job for incremental load.
918   */
919  @InterfaceAudience.Private
920  static Function<ColumnFamilyDescriptor, String> blockSizeDetails =
921    familyDescriptor -> String.valueOf(familyDescriptor.getBlocksize());
922
923  /**
924   * Serialize column family to bloom type map to configuration. Invoked while configuring the MR
925   * job for incremental load.
926   */
927  @InterfaceAudience.Private
928  static Function<ColumnFamilyDescriptor, String> bloomTypeDetails = familyDescriptor -> {
929    String bloomType = familyDescriptor.getBloomFilterType().toString();
930    if (bloomType == null) {
931      bloomType = ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
932    }
933    return bloomType;
934  };
935
936  /**
937   * Serialize column family to bloom param map to configuration. Invoked while configuring the MR
938   * job for incremental load.
939   */
940  @InterfaceAudience.Private
941  static Function<ColumnFamilyDescriptor, String> bloomParamDetails = familyDescriptor -> {
942    BloomType bloomType = familyDescriptor.getBloomFilterType();
943    String bloomParam = "";
944    if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
945      bloomParam = familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_LENGTH_KEY);
946    }
947    return bloomParam;
948  };
949
950  /**
951   * Serialize column family to data block encoding map to configuration. Invoked while configuring
952   * the MR job for incremental load.
953   */
954  @InterfaceAudience.Private
955  static Function<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
956    DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
957    if (encoding == null) {
958      encoding = DataBlockEncoding.NONE;
959    }
960    return encoding.toString();
961  };
962
963}