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