* @return The number of samples retrieved
*/
private long fullScanSampling(TablespaceSpec tablespace, final long sampleSize, Configuration hadoopConf,
Path outputPath, final int nSplits) throws TupleSamplerException {
MapOnlyJobBuilder builder = new MapOnlyJobBuilder(hadoopConf, "Reservoir Sampling to path " + outputPath);
for(Table table : tablespace.getPartitionedTables()) {
final TableSpec tableSpec = table.getTableSpec();
final String getPartitionByJavaScript = tableSpec.getPartitionByJavaScript();
for(TableInput inputFile : table.getFiles()) {
final RecordProcessor processor = inputFile.getRecordProcessor();
for(Path path : inputFile.getPaths()) {
builder.addInput(path, inputFile.getFormat(),
new MapOnlyMapper<ITuple, NullWritable, Text, NullWritable>() {
final int nSamples = (int) (sampleSize / nSplits);
final String[] samples = new String[nSamples];
CounterInterface counterInterface;
long recordCounter = 0;
JavascriptEngine jsEngine = null;
@Override
protected void setup(Context context, MultipleOutputsCollector coll) throws IOException,
InterruptedException {
counterInterface = new CounterInterface(context);
// Initialize JavaScript engine if needed
if (getPartitionByJavaScript != null) {
try {
jsEngine = new JavascriptEngine(getPartitionByJavaScript);
} catch(Throwable e) {
throw new RuntimeException(e);
}
}
};
// Collect Tuples with decreasing probability
// (http://en.wikipedia.org/wiki/Reservoir_sampling)
protected void map(ITuple key, NullWritable value, Context context) throws IOException,
InterruptedException {
ITuple uTuple;
try {
uTuple = processor.process(key, counterInterface);
} catch(Throwable e) {
throw new RuntimeException(e);
}
if(uTuple == null) { // user may have filtered the record
return;
}
long reservoirIndex;
if(recordCounter < nSamples) {
reservoirIndex = recordCounter;
} else {
reservoirIndex = (long) (Math.random() * recordCounter);
}
if(reservoirIndex < nSamples) {
String pkey = null;
try {
pkey = TablespaceGenerator.getPartitionByKey(
uTuple,
tableSpec,
jsEngine);
} catch (Throwable e) {
throw new RuntimeException("Error when determining partition key.", e);
}
samples[(int) reservoirIndex] = pkey;
}
recordCounter++;
}
// Write the in-memory sampled Tuples
protected void cleanup(Context context, MultipleOutputsCollector coll) throws IOException,
InterruptedException {
Text key = new Text();
for(String keyStr : samples) {
if(keyStr != null) {
key.set(keyStr);
context.write(key, NullWritable.get());
}
}
}
}, inputFile.getSpecificHadoopInputFormatContext());
}
}
}
// Set output path
Path outReservoirPath = new Path(outputPath + "-reservoir");
builder.setOutput(
outReservoirPath,
new HadoopOutputFormat(SequenceFileOutputFormat.class),
Text.class,
NullWritable.class);
builder.setJarByClass(callingClass);
try {
Job job = null;
job = builder.createJob();
if(!job.waitForCompletion(true)) {
throw new TupleSamplerException("Reservoir Sampling failed!");
}
} catch (Exception e) {
throw new TupleSamplerException("Error creating or launching the sampling job.", e);
}finally {
try {
builder.cleanUpInstanceFiles();
} catch (IOException e) {
throw new TupleSamplerException("Error cleaning up the sampling job.", e);
}
}