|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.hbase.util.bulkdatagenerator; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | +import java.math.BigDecimal; |
| 22 | +import java.util.List; |
| 23 | +import java.util.Map; |
| 24 | +import java.util.Random; |
| 25 | +import org.apache.commons.math3.util.Pair; |
| 26 | +import org.apache.hadoop.conf.Configuration; |
| 27 | +import org.apache.hadoop.hbase.KeyValue; |
| 28 | +import org.apache.hadoop.hbase.client.Admin; |
| 29 | +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; |
| 30 | +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; |
| 31 | +import org.apache.hadoop.io.NullWritable; |
| 32 | +import org.apache.hadoop.io.Text; |
| 33 | +import org.apache.hadoop.mapreduce.Mapper; |
| 34 | + |
| 35 | +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; |
| 36 | +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; |
| 37 | + |
| 38 | +public class BulkDataGeneratorMapper |
| 39 | + extends Mapper<Text, NullWritable, ImmutableBytesWritable, KeyValue> { |
| 40 | + |
| 41 | + /** Counter enumeration to count number of rows generated. */ |
| 42 | + public static enum Counters { |
| 43 | + ROWS_GENERATED |
| 44 | + } |
| 45 | + |
| 46 | + public static final String SPLIT_COUNT_KEY = |
| 47 | + BulkDataGeneratorMapper.class.getName() + "split.count"; |
| 48 | + |
| 49 | + private static final String ORG_ID = "00D000000000062"; |
| 50 | + private static final int MAX_EVENT_ID = Integer.MAX_VALUE; |
| 51 | + private static final int MAX_VEHICLE_ID = 100; |
| 52 | + private static final int MAX_SPEED_KPH = 140; |
| 53 | + private static final int NUM_LOCATIONS = 10; |
| 54 | + private static int splitCount = 1; |
| 55 | + private static final Random random = new Random(System.currentTimeMillis()); |
| 56 | + private static final Map<String, Pair<BigDecimal, BigDecimal>> LOCATIONS = |
| 57 | + Maps.newHashMapWithExpectedSize(NUM_LOCATIONS); |
| 58 | + private static final List<String> LOCATION_KEYS = Lists.newArrayListWithCapacity(NUM_LOCATIONS); |
| 59 | + static { |
| 60 | + LOCATIONS.put("Belém", new Pair<>(BigDecimal.valueOf(-01.45), BigDecimal.valueOf(-48.48))); |
| 61 | + LOCATIONS.put("Brasília", new Pair<>(BigDecimal.valueOf(-15.78), BigDecimal.valueOf(-47.92))); |
| 62 | + LOCATIONS.put("Campinas", new Pair<>(BigDecimal.valueOf(-22.90), BigDecimal.valueOf(-47.05))); |
| 63 | + LOCATIONS.put("Cuiaba", new Pair<>(BigDecimal.valueOf(-07.25), BigDecimal.valueOf(-58.42))); |
| 64 | + LOCATIONS.put("Manaus", new Pair<>(BigDecimal.valueOf(-03.10), BigDecimal.valueOf(-60.00))); |
| 65 | + LOCATIONS.put("Porto Velho", |
| 66 | + new Pair<>(BigDecimal.valueOf(-08.75), BigDecimal.valueOf(-63.90))); |
| 67 | + LOCATIONS.put("Recife", new Pair<>(BigDecimal.valueOf(-08.10), BigDecimal.valueOf(-34.88))); |
| 68 | + LOCATIONS.put("Rio de Janeiro", |
| 69 | + new Pair<>(BigDecimal.valueOf(-22.90), BigDecimal.valueOf(-43.23))); |
| 70 | + LOCATIONS.put("Santarém", new Pair<>(BigDecimal.valueOf(-02.43), BigDecimal.valueOf(-54.68))); |
| 71 | + LOCATIONS.put("São Paulo", new Pair<>(BigDecimal.valueOf(-23.53), BigDecimal.valueOf(-46.62))); |
| 72 | + LOCATION_KEYS.addAll(LOCATIONS.keySet()); |
| 73 | + } |
| 74 | + |
| 75 | + final static byte[] COLUMN_FAMILY_BYTES = Utility.COLUMN_FAMILY.getBytes(); |
| 76 | + |
| 77 | + /** {@inheritDoc} */ |
| 78 | + @Override |
| 79 | + protected void setup(Context context) throws IOException, InterruptedException { |
| 80 | + Configuration c = context.getConfiguration(); |
| 81 | + splitCount = c.getInt(SPLIT_COUNT_KEY, 1); |
| 82 | + } |
| 83 | + |
| 84 | + /** |
| 85 | + * Generates a single record based on value set to the key by |
| 86 | + * {@link BulkDataGeneratorRecordReader#getCurrentKey()}. |
| 87 | + * {@link Utility.TableColumnNames#TOOL_EVENT_ID} is first part of row key. Keeping first |
| 88 | + * {@link Utility#SPLIT_PREFIX_LENGTH} characters as index of the record to be generated ensures |
| 89 | + * that records are equally distributed across all regions of the table since region boundaries |
| 90 | + * are generated in similar fashion. Check {@link Utility#createTable(Admin, String, int, Map)} |
| 91 | + * method for region split info. |
| 92 | + * @param key - The key having index of next record to be generated |
| 93 | + * @param value - Value associated with the key (not used) |
| 94 | + * @param context - Context of the mapper container |
| 95 | + */ |
| 96 | + @Override |
| 97 | + protected void map(Text key, NullWritable value, Context context) |
| 98 | + throws IOException, InterruptedException { |
| 99 | + |
| 100 | + int recordIndex = Integer.parseInt(key.toString()); |
| 101 | + |
| 102 | + // <6-characters-region-boundary-prefix>_<15-random-chars>_<record-index-for-this-mapper-task> |
| 103 | + final String toolEventId = |
| 104 | + String.format("%0" + Utility.SPLIT_PREFIX_LENGTH + "d", recordIndex % (splitCount + 1)) + "_" |
| 105 | + + EnvironmentEdgeManager.currentTime() + (1e14 + (random.nextFloat() * 9e13)) + "_" |
| 106 | + + recordIndex; |
| 107 | + final String eventId = String.valueOf(Math.abs(random.nextInt(MAX_EVENT_ID))); |
| 108 | + final String vechileId = String.valueOf(Math.abs(random.nextInt(MAX_VEHICLE_ID))); |
| 109 | + final String speed = String.valueOf(Math.abs(random.nextInt(MAX_SPEED_KPH))); |
| 110 | + final String location = LOCATION_KEYS.get(random.nextInt(NUM_LOCATIONS)); |
| 111 | + final Pair<BigDecimal, BigDecimal> coordinates = LOCATIONS.get(location); |
| 112 | + final BigDecimal latitude = coordinates.getFirst(); |
| 113 | + final BigDecimal longitude = coordinates.getSecond(); |
| 114 | + |
| 115 | + final ImmutableBytesWritable hKey = |
| 116 | + new ImmutableBytesWritable(String.format("%s:%s", toolEventId, ORG_ID).getBytes()); |
| 117 | + addKeyValue(context, hKey, Utility.TableColumnNames.ORG_ID, ORG_ID); |
| 118 | + addKeyValue(context, hKey, Utility.TableColumnNames.TOOL_EVENT_ID, toolEventId); |
| 119 | + addKeyValue(context, hKey, Utility.TableColumnNames.EVENT_ID, eventId); |
| 120 | + addKeyValue(context, hKey, Utility.TableColumnNames.VEHICLE_ID, vechileId); |
| 121 | + addKeyValue(context, hKey, Utility.TableColumnNames.SPEED, speed); |
| 122 | + addKeyValue(context, hKey, Utility.TableColumnNames.LATITUDE, latitude.toString()); |
| 123 | + addKeyValue(context, hKey, Utility.TableColumnNames.LONGITUDE, longitude.toString()); |
| 124 | + addKeyValue(context, hKey, Utility.TableColumnNames.LOCATION, location); |
| 125 | + addKeyValue(context, hKey, Utility.TableColumnNames.TIMESTAMP, |
| 126 | + String.valueOf(EnvironmentEdgeManager.currentTime())); |
| 127 | + |
| 128 | + context.getCounter(Counters.ROWS_GENERATED).increment(1); |
| 129 | + } |
| 130 | + |
| 131 | + private void addKeyValue(final Context context, ImmutableBytesWritable key, |
| 132 | + final Utility.TableColumnNames columnName, final String value) |
| 133 | + throws IOException, InterruptedException { |
| 134 | + KeyValue kv = |
| 135 | + new KeyValue(key.get(), COLUMN_FAMILY_BYTES, columnName.getColumnName(), value.getBytes()); |
| 136 | + context.write(key, kv); |
| 137 | + } |
| 138 | +} |
0 commit comments