aboutsummaryrefslogtreecommitdiffstats
path: root/src/main/java/parallelai/spyglass/hbase/HBaseRecordReader.java
blob: 97077c40921b5493f195e6e9bfb58d4f1dadecb1 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
package parallelai.spyglass.hbase;

import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT;

import java.io.IOException;
import java.util.Set;
import java.util.TreeSet;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.util.StringUtils;

import parallelai.spyglass.hbase.HBaseConstants.SourceMode;


public class HBaseRecordReader implements RecordReader<ImmutableBytesWritable, Result> {

  static final Log LOG = LogFactory.getLog(HBaseRecordReader.class);

  private byte [] startRow;
  private byte [] endRow;
  private byte [] lastSuccessfulRow;
  private TreeSet<String> keyList;
  private SourceMode sourceMode;
  private Filter trrRowFilter;
  private ResultScanner scanner;
  private HTable htable;
  private byte [][] trrInputColumns;
  private long timestamp;
  private int rowcount;
  private boolean logScannerActivity = false;
  private int logPerRowCount = 100;
  private boolean endRowInclusive = true;

  /**
   * Restart from survivable exceptions by creating a new scanner.
   *
   * @param firstRow
   * @throws IOException
   */
  public void restartRangeScan(byte[] firstRow) throws IOException {
    Scan currentScan;
    if ((endRow != null) && (endRow.length > 0)) {
      if (trrRowFilter != null) {
        Scan scan = new Scan(firstRow, (endRowInclusive ?  
            Bytes.add(endRow, new byte[] {0}) : endRow ) );
        
        TableInputFormat.addColumns(scan, trrInputColumns);
        scan.setFilter(trrRowFilter);
        scan.setCacheBlocks(false);
        this.scanner = this.htable.getScanner(scan);
        currentScan = scan;
      } else {
        LOG.debug("TIFB.restart, firstRow: " +
            Bytes.toString(firstRow) + ", endRow: " +
            Bytes.toString(endRow));
        Scan scan = new Scan(firstRow, (endRowInclusive ? Bytes.add(endRow, new byte[] {0}) : endRow ));
        TableInputFormat.addColumns(scan, trrInputColumns);
        this.scanner = this.htable.getScanner(scan);
        currentScan = scan;
      }
    } else {
      LOG.debug("TIFB.restart, firstRow: " +
          Bytes.toStringBinary(firstRow) + ", no endRow");

      Scan scan = new Scan(firstRow);
      TableInputFormat.addColumns(scan, trrInputColumns);
      scan.setFilter(trrRowFilter);
      this.scanner = this.htable.getScanner(scan);
      currentScan = scan;
    }
    if (logScannerActivity) {
      LOG.info("Current scan=" + currentScan.toString());
      timestamp = System.currentTimeMillis();
      rowcount = 0;
    }
  }

  public TreeSet<String> getKeyList() {
    return keyList;
  }

  public void setKeyList(TreeSet<String> keyList) {
    this.keyList = keyList;
  }

  public SourceMode getSourceMode() {
    return sourceMode;
  }

  public void setSourceMode(SourceMode sourceMode) {
    this.sourceMode = sourceMode;
  }

  public byte[] getEndRow() {
    return endRow;
  }
  
  public void setEndRowInclusive(boolean isInclusive) {
    endRowInclusive = isInclusive;
  }
  
  public boolean getEndRowInclusive() {
    return endRowInclusive;
  }
  
  private byte [] nextKey = null;

  /**
   * Build the scanner. Not done in constructor to allow for extension.
   *
   * @throws IOException
   */
  public void init() throws IOException {
    switch( sourceMode ) {
      case SCAN_ALL:
      case SCAN_RANGE:
        restartRangeScan(startRow);
        break;
        
      case GET_LIST:
        nextKey = Bytes.toBytes(keyList.pollFirst());
        break;
        
      default:
        throw new IOException(" Unknown source mode : " + sourceMode );
    }
  }

  byte[] getStartRow() {
    return this.startRow;
  }
  /**
   * @param htable the {@link HTable} to scan.
   */
  public void setHTable(HTable htable) {
    Configuration conf = htable.getConfiguration();
    logScannerActivity = conf.getBoolean(
      ScannerCallable.LOG_SCANNER_ACTIVITY, false);
    logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
    this.htable = htable;
  }

  /**
   * @param inputColumns the columns to be placed in {@link Result}.
   */
  public void setInputColumns(final byte [][] inputColumns) {
    this.trrInputColumns = inputColumns;
  }

  /**
   * @param startRow the first row in the split
   */
  public void setStartRow(final byte [] startRow) {
    this.startRow = startRow;
  }

  /**
   *
   * @param endRow the last row in the split
   */
  public void setEndRow(final byte [] endRow) {
    this.endRow = endRow;
  }

  /**
   * @param rowFilter the {@link Filter} to be used.
   */
  public void setRowFilter(Filter rowFilter) {
    this.trrRowFilter = rowFilter;
  }

  @Override
  public void close() {
	  if (this.scanner != null) this.scanner.close();
  }

  /**
   * @return ImmutableBytesWritable
   *
   * @see org.apache.hadoop.mapred.RecordReader#createKey()
   */
  @Override
  public ImmutableBytesWritable createKey() {
    return new ImmutableBytesWritable();
  }

  /**
   * @return RowResult
   *
   * @see org.apache.hadoop.mapred.RecordReader#createValue()
   */
  @Override
  public Result createValue() {
    return new Result();
  }

  @Override
  public long getPos() {
    // This should be the ordinal tuple in the range;
    // not clear how to calculate...
    return 0;
  }

  @Override
  public float getProgress() {
    // Depends on the total number of tuples and getPos
    return 0;
  }

  /**
   * @param key HStoreKey as input key.
   * @param value MapWritable as input value
   * @return true if there was more data
   * @throws IOException
   */
  @Override
  public boolean next(ImmutableBytesWritable key, Result value)
  throws IOException {
    
    switch(sourceMode) {
      case SCAN_ALL:
      case SCAN_RANGE:
      {
        
        Result result;
        try {
          try {
            result = this.scanner.next();
            if (logScannerActivity) {
              rowcount ++;
              if (rowcount >= logPerRowCount) {
                long now = System.currentTimeMillis();
                LOG.info("Mapper took " + (now-timestamp)
                  + "ms to process " + rowcount + " rows");
                timestamp = now;
                rowcount = 0;
              }
            }
          } catch (IOException e) {
            // try to handle all IOExceptions by restarting
            // the scanner, if the second call fails, it will be rethrown
            LOG.debug("recovered from " + StringUtils.stringifyException(e));
            if (lastSuccessfulRow == null) {
              LOG.warn("We are restarting the first next() invocation," +
                  " if your mapper has restarted a few other times like this" +
                  " then you should consider killing this job and investigate" +
                  " why it's taking so long.");
            }
            if (lastSuccessfulRow == null) {
              restartRangeScan(startRow);
            } else {
              restartRangeScan(lastSuccessfulRow);
              this.scanner.next();    // skip presumed already mapped row
            }
            result = this.scanner.next();
          }

          if (result != null && result.size() > 0) {
            key.set(result.getRow());
            lastSuccessfulRow = key.get();
            Writables.copyWritable(result, value);
            return true;
          }
          return false;
        } catch (IOException ioe) {
          if (logScannerActivity) {
            long now = System.currentTimeMillis();
            LOG.info("Mapper took " + (now-timestamp)
              + "ms to process " + rowcount + " rows");
            LOG.info(ioe);
            String lastRow = lastSuccessfulRow == null ?
              "null" : Bytes.toStringBinary(lastSuccessfulRow);
            LOG.info("lastSuccessfulRow=" + lastRow);
          }
          throw ioe;
        }
      }
      
      case GET_LIST:
      {
        Result result;
        if( nextKey != null ) {
          result = this.htable.get(new Get(nextKey));
          
          if (result != null && result.size() > 0) {
        	System.out.println("KeyList => " + keyList);
        	System.out.println("Result => " + result);
        	if (keyList != null || !keyList.isEmpty()) {
        		
        		String newKey = keyList.pollFirst();
        		System.out.println("New Key => " + newKey);
        		nextKey = (newKey == null || newKey.length() == 0) ? null : Bytes.toBytes(newKey);
        	} else {
        		nextKey = null;
        	}
            key.set(result.getRow());
            lastSuccessfulRow = key.get();
            Writables.copyWritable(result, value);
            return true;
          }
          return false;
        } else {
          return false;
        }
      }
      
      default:
        throw new IOException("Unknown source mode : " + sourceMode );
    } 
  }
}