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 java.io.ByteArrayInputStream; 021import java.io.DataInput; 022import java.io.DataInputStream; 023import java.io.DataOutput; 024import java.io.IOException; 025import java.lang.reflect.InvocationTargetException; 026import java.lang.reflect.Method; 027import java.util.ArrayList; 028import java.util.Collections; 029import java.util.List; 030import java.util.Locale; 031import java.util.Map; 032import java.util.TreeMap; 033import java.util.UUID; 034import org.apache.hadoop.conf.Configuration; 035import org.apache.hadoop.conf.Configured; 036import org.apache.hadoop.fs.FileSystem; 037import org.apache.hadoop.fs.Path; 038import org.apache.hadoop.hbase.Cell; 039import org.apache.hadoop.hbase.CellComparator; 040import org.apache.hadoop.hbase.CellUtil; 041import org.apache.hadoop.hbase.HBaseConfiguration; 042import org.apache.hadoop.hbase.KeyValue; 043import org.apache.hadoop.hbase.KeyValueUtil; 044import org.apache.hadoop.hbase.PrivateCellUtil; 045import org.apache.hadoop.hbase.TableName; 046import org.apache.hadoop.hbase.Tag; 047import org.apache.hadoop.hbase.ZooKeeperConnectionException; 048import org.apache.hadoop.hbase.client.Admin; 049import org.apache.hadoop.hbase.client.Connection; 050import org.apache.hadoop.hbase.client.ConnectionFactory; 051import org.apache.hadoop.hbase.client.Delete; 052import org.apache.hadoop.hbase.client.Durability; 053import org.apache.hadoop.hbase.client.Mutation; 054import org.apache.hadoop.hbase.client.Put; 055import org.apache.hadoop.hbase.client.RegionLocator; 056import org.apache.hadoop.hbase.client.Result; 057import org.apache.hadoop.hbase.client.Table; 058import org.apache.hadoop.hbase.filter.Filter; 059import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 060import org.apache.hadoop.hbase.util.Bytes; 061import org.apache.hadoop.hbase.util.MapReduceExtendedCell; 062import org.apache.hadoop.hbase.zookeeper.ZKClusterId; 063import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 064import org.apache.hadoop.io.RawComparator; 065import org.apache.hadoop.io.WritableComparable; 066import org.apache.hadoop.io.WritableComparator; 067import org.apache.hadoop.mapreduce.Job; 068import org.apache.hadoop.mapreduce.Partitioner; 069import org.apache.hadoop.mapreduce.Reducer; 070import org.apache.hadoop.mapreduce.TaskCounter; 071import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; 072import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; 073import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; 074import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner; 075import org.apache.hadoop.util.Tool; 076import org.apache.hadoop.util.ToolRunner; 077import org.apache.yetus.audience.InterfaceAudience; 078import org.apache.zookeeper.KeeperException; 079import org.slf4j.Logger; 080import org.slf4j.LoggerFactory; 081 082/** 083 * Import data written by {@link Export}. 084 */ 085@InterfaceAudience.Public 086public class Import extends Configured implements Tool { 087 private static final Logger LOG = LoggerFactory.getLogger(Import.class); 088 final static String NAME = "import"; 089 public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS"; 090 public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output"; 091 public final static String FILTER_CLASS_CONF_KEY = "import.filter.class"; 092 public final static String FILTER_ARGS_CONF_KEY = "import.filter.args"; 093 public final static String TABLE_NAME = "import.table.name"; 094 public final static String WAL_DURABILITY = "import.wal.durability"; 095 public final static String HAS_LARGE_RESULT = "import.bulk.hasLargeResult"; 096 097 private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name"; 098 099 public static class CellWritableComparablePartitioner 100 extends Partitioner<CellWritableComparable, Cell> { 101 private static CellWritableComparable[] START_KEYS = null; 102 103 @Override 104 public int getPartition(CellWritableComparable key, Cell value, int numPartitions) { 105 for (int i = 0; i < START_KEYS.length; ++i) { 106 if (key.compareTo(START_KEYS[i]) <= 0) { 107 return i; 108 } 109 } 110 return START_KEYS.length; 111 } 112 113 } 114 115 /** 116 * @deprecated Use {@link CellWritableComparablePartitioner}. Will be removed from 3.0 onwards 117 */ 118 @Deprecated 119 public static class KeyValueWritableComparablePartitioner 120 extends Partitioner<KeyValueWritableComparable, KeyValue> { 121 private static KeyValueWritableComparable[] START_KEYS = null; 122 123 @Override 124 public int getPartition(KeyValueWritableComparable key, KeyValue value, int numPartitions) { 125 for (int i = 0; i < START_KEYS.length; ++i) { 126 if (key.compareTo(START_KEYS[i]) <= 0) { 127 return i; 128 } 129 } 130 return START_KEYS.length; 131 } 132 } 133 134 public static class KeyValueWritableComparable 135 implements WritableComparable<KeyValueWritableComparable> { 136 137 private KeyValue kv = null; 138 139 static { 140 // register this comparator 141 WritableComparator.define(KeyValueWritableComparable.class, new KeyValueWritableComparator()); 142 } 143 144 public KeyValueWritableComparable() { 145 } 146 147 public KeyValueWritableComparable(KeyValue kv) { 148 this.kv = kv; 149 } 150 151 @Override 152 public void write(DataOutput out) throws IOException { 153 KeyValue.write(kv, out); 154 } 155 156 @Override 157 public void readFields(DataInput in) throws IOException { 158 kv = KeyValue.create(in); 159 } 160 161 @Override 162 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS", 163 justification = "This is wrong, yes, but we should be purging Writables, not fixing them") 164 public int compareTo(KeyValueWritableComparable o) { 165 return CellComparator.getInstance().compare(this.kv, o.kv); 166 } 167 168 public static class KeyValueWritableComparator extends WritableComparator { 169 170 @Override 171 public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { 172 try { 173 KeyValueWritableComparable kv1 = new KeyValueWritableComparable(); 174 kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1))); 175 KeyValueWritableComparable kv2 = new KeyValueWritableComparable(); 176 kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2))); 177 return compare(kv1, kv2); 178 } catch (IOException e) { 179 throw new RuntimeException(e); 180 } 181 } 182 183 } 184 185 } 186 187 public static class CellWritableComparable implements WritableComparable<CellWritableComparable> { 188 189 private Cell kv = null; 190 191 static { 192 // register this comparator 193 WritableComparator.define(CellWritableComparable.class, new CellWritableComparator()); 194 } 195 196 public CellWritableComparable() { 197 } 198 199 public CellWritableComparable(Cell kv) { 200 this.kv = kv; 201 } 202 203 public Cell getCell() { 204 return kv; 205 } 206 207 @Override 208 public void write(DataOutput out) throws IOException { 209 int keyLen = PrivateCellUtil.estimatedSerializedSizeOfKey(kv); 210 int valueLen = 0; // We avoid writing value here. So just serialize as if an empty value. 211 out.writeInt(keyLen + valueLen + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE); 212 out.writeInt(keyLen); 213 out.writeInt(valueLen); 214 PrivateCellUtil.writeFlatKey(kv, out); 215 } 216 217 @Override 218 public void readFields(DataInput in) throws IOException { 219 kv = KeyValue.create(in); 220 } 221 222 @Override 223 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS", 224 justification = "This is wrong, yes, but we should be purging Writables, not fixing them") 225 public int compareTo(CellWritableComparable o) { 226 return CellComparator.getInstance().compare(this.kv, o.kv); 227 } 228 229 public static class CellWritableComparator extends WritableComparator { 230 231 @Override 232 public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { 233 try { 234 CellWritableComparable kv1 = new CellWritableComparable(); 235 kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1))); 236 CellWritableComparable kv2 = new CellWritableComparable(); 237 kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2))); 238 return compare(kv1, kv2); 239 } catch (IOException e) { 240 throw new RuntimeException(e); 241 } 242 } 243 244 } 245 246 } 247 248 /** 249 * @deprecated Use {@link CellReducer}. Will be removed from 3.0 onwards 250 */ 251 @Deprecated 252 public static class KeyValueReducer 253 extends Reducer<KeyValueWritableComparable, KeyValue, ImmutableBytesWritable, KeyValue> { 254 protected void reduce( 255 KeyValueWritableComparable row, Iterable<KeyValue> kvs, Reducer<KeyValueWritableComparable, 256 KeyValue, ImmutableBytesWritable, KeyValue>.Context context) 257 throws java.io.IOException, InterruptedException { 258 int index = 0; 259 for (KeyValue kv : kvs) { 260 context.write(new ImmutableBytesWritable(kv.getRowArray()), kv); 261 if (++index % 100 == 0) context.setStatus("Wrote " + index + " KeyValues, " 262 + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray())); 263 } 264 } 265 } 266 267 public static class CellReducer 268 extends Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell> { 269 protected void reduce(CellWritableComparable row, Iterable<Cell> kvs, 270 Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell>.Context context) 271 throws java.io.IOException, InterruptedException { 272 int index = 0; 273 for (Cell kv : kvs) { 274 context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), 275 new MapReduceExtendedCell(kv)); 276 if (++index % 100 == 0) context.setStatus("Wrote " + index + " KeyValues, " 277 + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray())); 278 } 279 } 280 } 281 282 /** 283 * @deprecated Use {@link CellSortImporter}. Will be removed from 3.0 onwards 284 */ 285 @Deprecated 286 public static class KeyValueSortImporter 287 extends TableMapper<KeyValueWritableComparable, KeyValue> { 288 private Map<byte[], byte[]> cfRenameMap; 289 private Filter filter; 290 private static final Logger LOG = LoggerFactory.getLogger(KeyValueSortImporter.class); 291 292 /** 293 * @param row The current table row key. 294 * @param value The columns. 295 * @param context The current context. 296 * @throws IOException When something is broken with the data. 297 */ 298 @Override 299 public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { 300 try { 301 if (LOG.isTraceEnabled()) { 302 LOG.trace( 303 "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength())); 304 } 305 if ( 306 filter == null || !filter.filterRowKey( 307 PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength())) 308 ) { 309 for (Cell kv : value.rawCells()) { 310 kv = filterKv(filter, kv); 311 // skip if we filtered it out 312 if (kv == null) continue; 313 // TODO get rid of ensureKeyValue 314 KeyValue ret = KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)); 315 context.write(new KeyValueWritableComparable(ret.createKeyOnly(false)), ret); 316 } 317 } 318 } catch (InterruptedException e) { 319 e.printStackTrace(); 320 } 321 } 322 323 @Override 324 public void setup(Context context) throws IOException { 325 cfRenameMap = createCfRenameMap(context.getConfiguration()); 326 filter = instantiateFilter(context.getConfiguration()); 327 int reduceNum = context.getNumReduceTasks(); 328 Configuration conf = context.getConfiguration(); 329 TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME)); 330 try (Connection conn = ConnectionFactory.createConnection(conf); 331 RegionLocator regionLocator = conn.getRegionLocator(tableName)) { 332 byte[][] startKeys = regionLocator.getStartKeys(); 333 if (startKeys.length != reduceNum) { 334 throw new IOException("Region split after job initialization"); 335 } 336 KeyValueWritableComparable[] startKeyWraps = 337 new KeyValueWritableComparable[startKeys.length - 1]; 338 for (int i = 1; i < startKeys.length; ++i) { 339 startKeyWraps[i - 1] = 340 new KeyValueWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i])); 341 } 342 KeyValueWritableComparablePartitioner.START_KEYS = startKeyWraps; 343 } 344 } 345 } 346 347 /** 348 * A mapper that just writes out KeyValues. 349 * @deprecated Use {@link CellImporter}. Will be removed from 3.0 onwards 350 */ 351 @Deprecated 352 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS", 353 justification = "Writables are going away and this has been this way forever") 354 public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> { 355 private Map<byte[], byte[]> cfRenameMap; 356 private Filter filter; 357 private static final Logger LOG = LoggerFactory.getLogger(KeyValueImporter.class); 358 359 /** 360 * @param row The current table row key. 361 * @param value The columns. 362 * @param context The current context. 363 * @throws IOException When something is broken with the data. 364 */ 365 @Override 366 public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { 367 try { 368 if (LOG.isTraceEnabled()) { 369 LOG.trace( 370 "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength())); 371 } 372 if ( 373 filter == null || !filter.filterRowKey( 374 PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength())) 375 ) { 376 for (Cell kv : value.rawCells()) { 377 kv = filterKv(filter, kv); 378 // skip if we filtered it out 379 if (kv == null) continue; 380 // TODO get rid of ensureKeyValue 381 context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap))); 382 } 383 } 384 } catch (InterruptedException e) { 385 e.printStackTrace(); 386 } 387 } 388 389 @Override 390 public void setup(Context context) { 391 cfRenameMap = createCfRenameMap(context.getConfiguration()); 392 filter = instantiateFilter(context.getConfiguration()); 393 } 394 } 395 396 public static class CellSortImporter extends TableMapper<CellWritableComparable, Cell> { 397 private Map<byte[], byte[]> cfRenameMap; 398 private Filter filter; 399 private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class); 400 401 /** 402 * @param row The current table row key. 403 * @param value The columns. 404 * @param context The current context. 405 * @throws IOException When something is broken with the data. 406 */ 407 @Override 408 public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { 409 try { 410 if (LOG.isTraceEnabled()) { 411 LOG.trace( 412 "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength())); 413 } 414 if ( 415 filter == null || !filter.filterRowKey( 416 PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength())) 417 ) { 418 for (Cell kv : value.rawCells()) { 419 kv = filterKv(filter, kv); 420 // skip if we filtered it out 421 if (kv == null) continue; 422 Cell ret = convertKv(kv, cfRenameMap); 423 context.write(new CellWritableComparable(ret), new MapReduceExtendedCell(ret)); 424 } 425 } 426 } catch (InterruptedException e) { 427 e.printStackTrace(); 428 } 429 } 430 431 @Override 432 public void setup(Context context) throws IOException { 433 cfRenameMap = createCfRenameMap(context.getConfiguration()); 434 filter = instantiateFilter(context.getConfiguration()); 435 int reduceNum = context.getNumReduceTasks(); 436 Configuration conf = context.getConfiguration(); 437 TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME)); 438 try (Connection conn = ConnectionFactory.createConnection(conf); 439 RegionLocator regionLocator = conn.getRegionLocator(tableName)) { 440 byte[][] startKeys = regionLocator.getStartKeys(); 441 if (startKeys.length != reduceNum) { 442 throw new IOException("Region split after job initialization"); 443 } 444 CellWritableComparable[] startKeyWraps = new CellWritableComparable[startKeys.length - 1]; 445 for (int i = 1; i < startKeys.length; ++i) { 446 startKeyWraps[i - 1] = 447 new CellWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i])); 448 } 449 CellWritableComparablePartitioner.START_KEYS = startKeyWraps; 450 } 451 } 452 } 453 454 /** 455 * A mapper that just writes out KeyValues. 456 */ 457 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS", 458 justification = "Writables are going away and this has been this way forever") 459 public static class CellImporter extends TableMapper<ImmutableBytesWritable, Cell> { 460 private Map<byte[], byte[]> cfRenameMap; 461 private Filter filter; 462 private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class); 463 464 /** 465 * @param row The current table row key. 466 * @param value The columns. 467 * @param context The current context. 468 * @throws IOException When something is broken with the data. 469 */ 470 @Override 471 public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { 472 try { 473 if (LOG.isTraceEnabled()) { 474 LOG.trace( 475 "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength())); 476 } 477 if ( 478 filter == null || !filter.filterRowKey( 479 PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength())) 480 ) { 481 for (Cell kv : value.rawCells()) { 482 kv = filterKv(filter, kv); 483 // skip if we filtered it out 484 if (kv == null) continue; 485 context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap))); 486 } 487 } 488 } catch (InterruptedException e) { 489 e.printStackTrace(); 490 } 491 } 492 493 @Override 494 public void setup(Context context) { 495 cfRenameMap = createCfRenameMap(context.getConfiguration()); 496 filter = instantiateFilter(context.getConfiguration()); 497 } 498 } 499 500 /** 501 * Write table content out to files in hdfs. 502 */ 503 public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> { 504 private Map<byte[], byte[]> cfRenameMap; 505 private List<UUID> clusterIds; 506 private Filter filter; 507 private Durability durability; 508 509 /** 510 * @param row The current table row key. 511 * @param value The columns. 512 * @param context The current context. 513 * @throws IOException When something is broken with the data. 514 */ 515 @Override 516 public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException { 517 try { 518 writeResult(row, value, context); 519 } catch (InterruptedException e) { 520 e.printStackTrace(); 521 } 522 } 523 524 private void writeResult(ImmutableBytesWritable key, Result result, Context context) 525 throws IOException, InterruptedException { 526 Put put = null; 527 Delete delete = null; 528 if (LOG.isTraceEnabled()) { 529 LOG.trace( 530 "Considering the row." + Bytes.toString(key.get(), key.getOffset(), key.getLength())); 531 } 532 if ( 533 filter == null || !filter.filterRowKey( 534 PrivateCellUtil.createFirstOnRow(key.get(), key.getOffset(), (short) key.getLength())) 535 ) { 536 processKV(key, result, context, put, delete); 537 } 538 } 539 540 protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put, 541 Delete delete) throws IOException, InterruptedException { 542 for (Cell kv : result.rawCells()) { 543 kv = filterKv(filter, kv); 544 // skip if we filter it out 545 if (kv == null) continue; 546 547 kv = convertKv(kv, cfRenameMap); 548 // Deletes and Puts are gathered and written when finished 549 /* 550 * If there are sequence of mutations and tombstones in an Export, and after Import the same 551 * sequence should be restored as it is. If we combine all Delete tombstones into single 552 * request then there is chance of ignoring few DeleteFamily tombstones, because if we 553 * submit multiple DeleteFamily tombstones in single Delete request then we are maintaining 554 * only newest in hbase table and ignoring other. Check - HBASE-12065 555 */ 556 if (PrivateCellUtil.isDeleteFamily(kv)) { 557 Delete deleteFamily = new Delete(key.get()); 558 deleteFamily.add(kv); 559 if (durability != null) { 560 deleteFamily.setDurability(durability); 561 } 562 deleteFamily.setClusterIds(clusterIds); 563 context.write(key, deleteFamily); 564 } else if (CellUtil.isDelete(kv)) { 565 if (delete == null) { 566 delete = new Delete(key.get()); 567 } 568 delete.add(kv); 569 } else { 570 if (put == null) { 571 put = new Put(key.get()); 572 } 573 addPutToKv(put, kv); 574 } 575 } 576 if (put != null) { 577 if (durability != null) { 578 put.setDurability(durability); 579 } 580 put.setClusterIds(clusterIds); 581 context.write(key, put); 582 } 583 if (delete != null) { 584 if (durability != null) { 585 delete.setDurability(durability); 586 } 587 delete.setClusterIds(clusterIds); 588 context.write(key, delete); 589 } 590 } 591 592 protected void addPutToKv(Put put, Cell kv) throws IOException { 593 put.add(kv); 594 } 595 596 @Override 597 public void setup(Context context) { 598 LOG.info("Setting up " + getClass() + " mapper."); 599 Configuration conf = context.getConfiguration(); 600 cfRenameMap = createCfRenameMap(conf); 601 filter = instantiateFilter(conf); 602 String durabilityStr = conf.get(WAL_DURABILITY); 603 if (durabilityStr != null) { 604 durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT)); 605 LOG.info("setting WAL durability to " + durability); 606 } else { 607 LOG.info("setting WAL durability to default."); 608 } 609 // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid. 610 ZKWatcher zkw = null; 611 Exception ex = null; 612 try { 613 zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null); 614 clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw)); 615 } catch (ZooKeeperConnectionException e) { 616 ex = e; 617 LOG.error("Problem connecting to ZooKeper during task setup", e); 618 } catch (KeeperException e) { 619 ex = e; 620 LOG.error("Problem reading ZooKeeper data during task setup", e); 621 } catch (IOException e) { 622 ex = e; 623 LOG.error("Problem setting up task", e); 624 } finally { 625 if (zkw != null) zkw.close(); 626 } 627 if (clusterIds == null) { 628 // exit early if setup fails 629 throw new RuntimeException(ex); 630 } 631 } 632 } 633 634 /** 635 * Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to 636 * optionally not include in the job output 637 * @param conf {@link Configuration} from which to load the filter 638 * @return the filter to use for the task, or <tt>null</tt> if no filter to should be used 639 * @throws IllegalArgumentException if the filter is misconfigured 640 */ 641 public static Filter instantiateFilter(Configuration conf) { 642 // get the filter, if it was configured 643 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class); 644 if (filterClass == null) { 645 LOG.debug("No configured filter class, accepting all keyvalues."); 646 return null; 647 } 648 LOG.debug("Attempting to create filter:" + filterClass); 649 String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY); 650 ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs); 651 try { 652 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class); 653 return (Filter) m.invoke(null, quotedArgs); 654 } catch (IllegalAccessException e) { 655 LOG.error("Couldn't instantiate filter!", e); 656 throw new RuntimeException(e); 657 } catch (SecurityException e) { 658 LOG.error("Couldn't instantiate filter!", e); 659 throw new RuntimeException(e); 660 } catch (NoSuchMethodException e) { 661 LOG.error("Couldn't instantiate filter!", e); 662 throw new RuntimeException(e); 663 } catch (IllegalArgumentException e) { 664 LOG.error("Couldn't instantiate filter!", e); 665 throw new RuntimeException(e); 666 } catch (InvocationTargetException e) { 667 LOG.error("Couldn't instantiate filter!", e); 668 throw new RuntimeException(e); 669 } 670 } 671 672 private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) { 673 ArrayList<byte[]> quotedArgs = new ArrayList<>(); 674 for (String stringArg : stringArgs) { 675 // all the filters' instantiation methods expected quoted args since they are coming from 676 // the shell, so add them here, though it shouldn't really be needed :-/ 677 quotedArgs.add(Bytes.toBytes("'" + stringArg + "'")); 678 } 679 return quotedArgs; 680 } 681 682 /** 683 * Attempt to filter out the keyvalue 684 * @param c {@link Cell} on which to apply the filter 685 * @return <tt>null</tt> if the key should not be written, otherwise returns the original 686 * {@link Cell} 687 */ 688 public static Cell filterKv(Filter filter, Cell c) throws IOException { 689 // apply the filter and skip this kv if the filter doesn't apply 690 if (filter != null) { 691 Filter.ReturnCode code = filter.filterCell(c); 692 if (LOG.isTraceEnabled()) { 693 LOG.trace("Filter returned:" + code + " for the cell:" + c); 694 } 695 // if its not an accept type, then skip this kv 696 if ( 697 !(code.equals(Filter.ReturnCode.INCLUDE) 698 || code.equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL)) 699 ) { 700 return null; 701 } 702 } 703 return c; 704 } 705 706 // helper: create a new KeyValue based on CF rename map 707 private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) { 708 if (cfRenameMap != null) { 709 // If there's a rename mapping for this CF, create a new KeyValue 710 byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv)); 711 if (newCfName != null) { 712 List<Tag> tags = PrivateCellUtil.getTags(kv); 713 kv = new KeyValue(kv.getRowArray(), // row buffer 714 kv.getRowOffset(), // row offset 715 kv.getRowLength(), // row length 716 newCfName, // CF buffer 717 0, // CF offset 718 newCfName.length, // CF length 719 kv.getQualifierArray(), // qualifier buffer 720 kv.getQualifierOffset(), // qualifier offset 721 kv.getQualifierLength(), // qualifier length 722 kv.getTimestamp(), // timestamp 723 KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type 724 kv.getValueArray(), // value buffer 725 kv.getValueOffset(), // value offset 726 kv.getValueLength(), // value length 727 tags.size() == 0 ? null : tags); 728 } 729 } 730 return kv; 731 } 732 733 // helper: make a map from sourceCfName to destCfName by parsing a config key 734 private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) { 735 Map<byte[], byte[]> cfRenameMap = null; 736 String allMappingsPropVal = conf.get(CF_RENAME_PROP); 737 if (allMappingsPropVal != null) { 738 // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,... 739 String[] allMappings = allMappingsPropVal.split(","); 740 for (String mapping : allMappings) { 741 if (cfRenameMap == null) { 742 cfRenameMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); 743 } 744 String[] srcAndDest = mapping.split(":"); 745 if (srcAndDest.length != 2) { 746 continue; 747 } 748 cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes()); 749 } 750 } 751 return cfRenameMap; 752 } 753 754 /** 755 * <p> 756 * Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells the mapper 757 * how to rename column families. 758 * <p> 759 * Alternately, instead of calling this function, you could set the configuration key 760 * {@link #CF_RENAME_PROP} yourself. The value should look like 761 * 762 * <pre> 763 * srcCf1:destCf1,srcCf2:destCf2,.... 764 * </pre> 765 * 766 * . This would have the same effect on the mapper behavior. 767 * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be set 768 * @param renameMap a mapping from source CF names to destination CF names 769 */ 770 static public void configureCfRenaming(Configuration conf, Map<String, String> renameMap) { 771 StringBuilder sb = new StringBuilder(); 772 for (Map.Entry<String, String> entry : renameMap.entrySet()) { 773 String sourceCf = entry.getKey(); 774 String destCf = entry.getValue(); 775 776 if ( 777 sourceCf.contains(":") || sourceCf.contains(",") || destCf.contains(":") 778 || destCf.contains(",") 779 ) { 780 throw new IllegalArgumentException( 781 "Illegal character in CF names: " + sourceCf + ", " + destCf); 782 } 783 784 if (sb.length() != 0) { 785 sb.append(","); 786 } 787 sb.append(sourceCf + ":" + destCf); 788 } 789 conf.set(CF_RENAME_PROP, sb.toString()); 790 } 791 792 /** 793 * Add a Filter to be instantiated on import 794 * @param conf Configuration to update (will be passed to the job) 795 * @param clazz {@link Filter} subclass to instantiate on the server. 796 * @param filterArgs List of arguments to pass to the filter on instantiation 797 */ 798 public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz, 799 List<String> filterArgs) throws IOException { 800 conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName()); 801 conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()])); 802 } 803 804 /** 805 * Sets up the actual job. 806 * @param conf The current configuration. 807 * @param args The command line parameters. 808 * @return The newly created job. 809 * @throws IOException When setting up the job fails. 810 */ 811 public static Job createSubmittableJob(Configuration conf, String[] args) throws IOException { 812 TableName tableName = TableName.valueOf(args[0]); 813 conf.set(TABLE_NAME, tableName.getNameAsString()); 814 Path inputDir = new Path(args[1]); 815 Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName)); 816 job.setJarByClass(Importer.class); 817 FileInputFormat.setInputPaths(job, inputDir); 818 job.setInputFormatClass(SequenceFileInputFormat.class); 819 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); 820 821 // make sure we get the filter in the jars 822 try { 823 Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class); 824 if (filter != null) { 825 TableMapReduceUtil.addDependencyJarsForClasses(conf, filter); 826 } 827 } catch (Exception e) { 828 throw new IOException(e); 829 } 830 831 if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) { 832 LOG.info("Use Large Result!!"); 833 try (Connection conn = ConnectionFactory.createConnection(conf); 834 Table table = conn.getTable(tableName); 835 RegionLocator regionLocator = conn.getRegionLocator(tableName)) { 836 HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); 837 job.setMapperClass(CellSortImporter.class); 838 job.setReducerClass(CellReducer.class); 839 Path outputDir = new Path(hfileOutPath); 840 FileOutputFormat.setOutputPath(job, outputDir); 841 job.setMapOutputKeyClass(CellWritableComparable.class); 842 job.setMapOutputValueClass(MapReduceExtendedCell.class); 843 job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class", 844 CellWritableComparable.CellWritableComparator.class, RawComparator.class); 845 Path partitionsPath = 846 new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration())); 847 FileSystem fs = FileSystem.get(job.getConfiguration()); 848 fs.deleteOnExit(partitionsPath); 849 job.setPartitionerClass(CellWritableComparablePartitioner.class); 850 job.setNumReduceTasks(regionLocator.getStartKeys().length); 851 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), 852 org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class); 853 } 854 } else if (hfileOutPath != null) { 855 LOG.info("writing to hfiles for bulk load."); 856 job.setMapperClass(CellImporter.class); 857 try (Connection conn = ConnectionFactory.createConnection(conf); 858 Table table = conn.getTable(tableName); 859 RegionLocator regionLocator = conn.getRegionLocator(tableName)) { 860 job.setReducerClass(CellSortReducer.class); 861 Path outputDir = new Path(hfileOutPath); 862 FileOutputFormat.setOutputPath(job, outputDir); 863 job.setMapOutputKeyClass(ImmutableBytesWritable.class); 864 job.setMapOutputValueClass(MapReduceExtendedCell.class); 865 HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator); 866 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), 867 org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class); 868 } 869 } else { 870 LOG.info("writing directly to table from Mapper."); 871 // No reducers. Just write straight to table. Call initTableReducerJob 872 // because it sets up the TableOutputFormat. 873 job.setMapperClass(Importer.class); 874 TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job); 875 job.setNumReduceTasks(0); 876 } 877 return job; 878 } 879 880 /* 881 * @param errorMsg Error message. Can be null. 882 */ 883 private static void usage(final String errorMsg) { 884 if (errorMsg != null && errorMsg.length() > 0) { 885 System.err.println("ERROR: " + errorMsg); 886 } 887 System.err.println("Usage: Import [options] <tablename> <inputdir>"); 888 System.err.println("By default Import will load data directly into HBase. To instead generate"); 889 System.err.println("HFiles of data to prepare for a bulk data load, pass the option:"); 890 System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output"); 891 System.err.println("If there is a large result that includes too much Cell " 892 + "whitch can occur OOME caused by the memery sort in reducer, pass the option:"); 893 System.err.println(" -D" + HAS_LARGE_RESULT + "=true"); 894 System.err 895 .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use"); 896 System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>"); 897 System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter"); 898 System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the " 899 + CF_RENAME_PROP + " property. Futher, filters will only use the" 900 + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify " 901 + " whether the current row needs to be ignored completely for processing and " 902 + " Filter#filterCell(Cell) method to determine if the Cell should be added;" 903 + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including" 904 + " the Cell."); 905 System.err.println("To import data exported from HBase 0.94, use"); 906 System.err.println(" -Dhbase.import.version=0.94"); 907 System.err.println(" -D " + JOB_NAME_CONF_KEY 908 + "=jobName - use the specified mapreduce job name for the import"); 909 System.err.println("For performance consider the following options:\n" 910 + " -Dmapreduce.map.speculative=false\n" + " -Dmapreduce.reduce.speculative=false\n" 911 + " -D" + WAL_DURABILITY + "=<Used while writing data to hbase." 912 + " Allowed values are the supported durability values" 913 + " like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>"); 914 } 915 916 /** 917 * If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we 918 * need to flush all the regions of the table as the data is held in memory and is also not 919 * present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the 920 * regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL} 921 */ 922 public static void flushRegionsIfNecessary(Configuration conf) 923 throws IOException, InterruptedException { 924 String tableName = conf.get(TABLE_NAME); 925 Admin hAdmin = null; 926 Connection connection = null; 927 String durability = conf.get(WAL_DURABILITY); 928 // Need to flush if the data is written to hbase and skip wal is enabled. 929 if ( 930 conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null 931 && Durability.SKIP_WAL.name().equalsIgnoreCase(durability) 932 ) { 933 LOG.info("Flushing all data that skipped the WAL."); 934 try { 935 connection = ConnectionFactory.createConnection(conf); 936 hAdmin = connection.getAdmin(); 937 hAdmin.flush(TableName.valueOf(tableName)); 938 } finally { 939 if (hAdmin != null) { 940 hAdmin.close(); 941 } 942 if (connection != null) { 943 connection.close(); 944 } 945 } 946 } 947 } 948 949 @Override 950 public int run(String[] args) throws Exception { 951 if (args.length < 2) { 952 usage("Wrong number of arguments: " + args.length); 953 return -1; 954 } 955 String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER); 956 if (inputVersionString != null) { 957 getConf().set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString); 958 } 959 Job job = createSubmittableJob(getConf(), args); 960 boolean isJobSuccessful = job.waitForCompletion(true); 961 if (isJobSuccessful) { 962 // Flush all the regions of the table 963 flushRegionsIfNecessary(getConf()); 964 } 965 long inputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue(); 966 long outputRecords = job.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS).getValue(); 967 if (outputRecords < inputRecords) { 968 System.err.println("Warning, not all records were imported (maybe filtered out)."); 969 if (outputRecords == 0) { 970 System.err.println("If the data was exported from HBase 0.94 " 971 + "consider using -Dhbase.import.version=0.94."); 972 } 973 } 974 975 return (isJobSuccessful ? 0 : 1); 976 } 977 978 /** 979 * Main entry point. 980 * @param args The command line parameters. 981 * @throws Exception When running the job fails. 982 */ 983 public static void main(String[] args) throws Exception { 984 int errCode = ToolRunner.run(HBaseConfiguration.create(), new Import(), args); 985 System.exit(errCode); 986 } 987 988}