This repository has been archived by the owner on Apr 9, 2020. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 3
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
ACCUMULO-381 added a bulk ingest option for wikisearch ingest
git-svn-id: https://svn.apache.org/repos/asf/incubator/accumulo/branches/1.4@1243506 13f79535-47bb-0310-9956-ffa450edef68
- Loading branch information
Adam Fuchs
committed
Feb 13, 2012
1 parent
72b7221
commit b4f3087
Showing
5 changed files
with
314 additions
and
12 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
129 changes: 129 additions & 0 deletions
129
.../main/java/org/apache/accumulo/examples/wikisearch/output/BufferingRFileRecordWriter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
package org.apache.accumulo.examples.wikisearch.output; | ||
|
||
import java.io.IOException; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.TreeMap; | ||
import java.util.Map.Entry; | ||
|
||
import org.apache.accumulo.core.conf.AccumuloConfiguration; | ||
import org.apache.accumulo.core.data.ColumnUpdate; | ||
import org.apache.accumulo.core.data.Key; | ||
import org.apache.accumulo.core.data.Mutation; | ||
import org.apache.accumulo.core.data.Value; | ||
import org.apache.accumulo.core.file.FileSKVWriter; | ||
import org.apache.accumulo.core.file.rfile.RFileOperations; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.io.Text; | ||
import org.apache.hadoop.mapreduce.RecordWriter; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
|
||
final class BufferingRFileRecordWriter extends RecordWriter<Text,Mutation> { | ||
private final long maxSize; | ||
private final AccumuloConfiguration acuconf; | ||
private final Configuration conf; | ||
private final String filenamePrefix; | ||
private final String taskID; | ||
private final FileSystem fs; | ||
private int fileCount = 0; | ||
private long size; | ||
|
||
private Map<Text,TreeMap<Key,Value>> buffers = new HashMap<Text,TreeMap<Key,Value>>(); | ||
private Map<Text,Long> bufferSizes = new HashMap<Text,Long>(); | ||
|
||
private TreeMap<Key,Value> getBuffer(Text tablename) | ||
{ | ||
TreeMap<Key,Value> buffer = buffers.get(tablename); | ||
if(buffer == null) | ||
{ | ||
buffer = new TreeMap<Key,Value>(); | ||
buffers.put(tablename, buffer); | ||
bufferSizes.put(tablename, 0l); | ||
} | ||
return buffer; | ||
} | ||
|
||
private Text getLargestTablename() | ||
{ | ||
long max = 0; | ||
Text table = null; | ||
for(Entry<Text,Long> e:bufferSizes.entrySet()) | ||
{ | ||
if(e.getValue() > max) | ||
{ | ||
max = e.getValue(); | ||
table = e.getKey(); | ||
} | ||
} | ||
return table; | ||
} | ||
|
||
private void flushLargestTable() throws IOException | ||
{ | ||
Text tablename = getLargestTablename(); | ||
if(tablename == null) | ||
return; | ||
long bufferSize = bufferSizes.get(tablename); | ||
TreeMap<Key,Value> buffer = buffers.get(tablename); | ||
if (buffer.size() == 0) | ||
return; | ||
|
||
// TODO fix the filename | ||
String file = filenamePrefix + "/" + tablename + "/" + taskID + "_" + (fileCount++) + ".rf"; | ||
FileSKVWriter writer = RFileOperations.getInstance().openWriter(file, fs, conf, acuconf); | ||
|
||
// forget locality groups for now, just write everything to the default | ||
writer.startDefaultLocalityGroup(); | ||
|
||
for (Entry<Key,Value> e : buffer.entrySet()) { | ||
writer.append(e.getKey(), e.getValue()); | ||
} | ||
|
||
writer.close(); | ||
|
||
size -= bufferSize; | ||
buffer.clear(); | ||
bufferSizes.put(tablename, 0l); | ||
} | ||
|
||
BufferingRFileRecordWriter(long maxSize, AccumuloConfiguration acuconf, Configuration conf, String filenamePrefix, String taskID, FileSystem fs) { | ||
this.maxSize = maxSize; | ||
this.acuconf = acuconf; | ||
this.conf = conf; | ||
this.filenamePrefix = filenamePrefix; | ||
this.taskID = taskID; | ||
this.fs = fs; | ||
} | ||
|
||
@Override | ||
public void close(TaskAttemptContext arg0) throws IOException, InterruptedException { | ||
while(size > 0) | ||
flushLargestTable(); | ||
} | ||
|
||
@Override | ||
public void write(Text table, Mutation mutation) throws IOException, InterruptedException { | ||
TreeMap<Key,Value> buffer = getBuffer(table); | ||
int mutationSize = 0; | ||
for(ColumnUpdate update: mutation.getUpdates()) | ||
{ | ||
Key k = new Key(mutation.getRow(),update.getColumnFamily(),update.getColumnQualifier(),update.getColumnVisibility(),update.getTimestamp(),update.isDeleted()); | ||
Value v = new Value(update.getValue()); | ||
mutationSize += k.getSize(); | ||
mutationSize += v.getSize(); | ||
buffer.put(k, v); | ||
} | ||
size += mutationSize; | ||
long bufferSize = bufferSizes.get(table); | ||
bufferSize += mutationSize; | ||
bufferSizes.put(table, bufferSize); | ||
|
||
// TODO add object overhead size | ||
|
||
while (size >= maxSize) { | ||
flushLargestTable(); | ||
} | ||
} | ||
|
||
} |
103 changes: 103 additions & 0 deletions
103
...rc/main/java/org/apache/accumulo/examples/wikisearch/output/SortingRFileOutputFormat.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
package org.apache.accumulo.examples.wikisearch.output; | ||
|
||
import java.io.IOException; | ||
|
||
import org.apache.accumulo.core.conf.AccumuloConfiguration; | ||
import org.apache.accumulo.core.data.Mutation; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.mapreduce.JobContext; | ||
import org.apache.hadoop.mapreduce.OutputCommitter; | ||
import org.apache.hadoop.mapreduce.OutputFormat; | ||
import org.apache.hadoop.mapreduce.RecordWriter; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.io.Text; | ||
|
||
public class SortingRFileOutputFormat extends OutputFormat<Text,Mutation> { | ||
|
||
public static final String PATH_NAME = "sortingrfileoutputformat.path"; | ||
public static final String MAX_BUFFER_SIZE = "sortingrfileoutputformat.max.buffer.size"; | ||
|
||
public static void setPathName(Configuration conf, String path) { | ||
conf.set(PATH_NAME, path); | ||
} | ||
|
||
public static String getPathName(Configuration conf) { | ||
return conf.get(PATH_NAME); | ||
} | ||
|
||
public static void setMaxBufferSize(Configuration conf, long maxBufferSize) { | ||
conf.setLong(MAX_BUFFER_SIZE, maxBufferSize); | ||
} | ||
|
||
public static long getMaxBufferSize(Configuration conf) { | ||
return conf.getLong(MAX_BUFFER_SIZE, -1); | ||
} | ||
|
||
@Override | ||
public void checkOutputSpecs(JobContext job) throws IOException, InterruptedException { | ||
// TODO make sure the path is writable? | ||
// TODO make sure the max buffer size is set and is reasonable | ||
} | ||
|
||
@Override | ||
public OutputCommitter getOutputCommitter(TaskAttemptContext arg0) throws IOException, InterruptedException { | ||
return new OutputCommitter() { | ||
|
||
@Override | ||
public void setupTask(TaskAttemptContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
|
||
} | ||
|
||
@Override | ||
public void setupJob(JobContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
|
||
} | ||
|
||
@Override | ||
public boolean needsTaskCommit(TaskAttemptContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
return false; | ||
} | ||
|
||
@Override | ||
public void commitTask(TaskAttemptContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
|
||
} | ||
|
||
@Override | ||
public void cleanupJob(JobContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
|
||
} | ||
|
||
@Override | ||
public void abortTask(TaskAttemptContext arg0) throws IOException { | ||
// TODO Auto-generated method stub | ||
|
||
} | ||
}; | ||
} | ||
|
||
@Override | ||
public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt) throws IOException, InterruptedException { | ||
|
||
// grab the configuration | ||
final Configuration conf = attempt.getConfiguration(); | ||
// create a filename | ||
final String filenamePrefix = getPathName(conf); | ||
final String taskID = attempt.getTaskAttemptID().toString(); | ||
// grab the max size | ||
final long maxSize = getMaxBufferSize(conf); | ||
// grab the FileSystem | ||
final FileSystem fs = FileSystem.get(conf); | ||
// create a default AccumuloConfiguration | ||
final AccumuloConfiguration acuconf = AccumuloConfiguration.getDefaultConfiguration(); | ||
|
||
return new BufferingRFileRecordWriter(maxSize, acuconf, conf, filenamePrefix, taskID, fs); | ||
} | ||
|
||
} |