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