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}