diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java new file mode 100644 index 0000000000..b0cfb63479 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java @@ -0,0 +1,321 @@ +package voldemort.store.readonly.disk; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +public class HadoopStoreWriter implements KeyValueWriter { + + private static final Logger logger = Logger.getLogger(HadoopStoreWriter.class); + + private DataOutputStream indexFileStream = null; + private DataOutputStream valueFileStream = null; + private int position; + private String taskId = null; + + private int nodeId = -1; + private int partitionId = -1; + private int chunkId = -1; + private int replicaType = -1; + + private Path taskIndexFileName; + private Path taskValueFileName; + + private JobConf conf; + private CheckSumType checkSumType; + private CheckSum checkSumDigestIndex; + private CheckSum checkSumDigestValue; + + private String outputDir; + + private FileSystem fs; + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + + @Override + public void conf(JobConf job) { + + conf = job; + try { + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + this.conf = job; + this.position = 0; + this.outputDir = job.get("final.output.dir"); + this.taskId = job.get("mapred.task.id"); + this.checkSumType = CheckSum.fromString(job.get("checksum.type")); + this.checkSumDigestIndex = CheckSum.getInstance(checkSumType); + this.checkSumDigestValue = CheckSum.getInstance(checkSumType); + + this.taskIndexFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() + + "." + + this.taskId + + ".index"); + this.taskValueFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() + + "." + + this.taskId + + ".data"); + + if(this.fs == null) + this.fs = this.taskIndexFileName.getFileSystem(job); + + this.indexFileStream = fs.create(this.taskIndexFileName); + this.valueFileStream = fs.create(this.taskValueFileName); + + logger.info("Opening " + this.taskIndexFileName + " and " + this.taskValueFileName + + " for writing."); + + } catch(IOException e) { + throw new RuntimeException("Failed to open Input/OutputStream", e); + } + + } + + @Override + public void write(BytesWritable key, Iterator iterator, Reporter reporter) + throws IOException { + + // Write key and position + this.indexFileStream.write(key.get(), 0, key.getSize()); + this.indexFileStream.writeInt(this.position); + + // Run key through checksum digest + if(this.checkSumDigestIndex != null) { + this.checkSumDigestIndex.update(key.get(), 0, key.getSize()); + this.checkSumDigestIndex.update(this.position); + } + + short numTuples = 0; + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream valueStream = new DataOutputStream(stream); + + while(iterator.hasNext()) { + BytesWritable writable = iterator.next(); + byte[] valueBytes = writable.get(); + int offsetTillNow = 0; + + // Read node Id + if(this.nodeId == -1) + this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read partition id + if(this.partitionId == -1) + this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read chunk id + if(this.chunkId == -1) + this.chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); + + // Read replica type + if(getSaveKeys()) { + if(this.replicaType == -1) + this.replicaType = (int) ByteUtils.readBytes(valueBytes, + offsetTillNow, + ByteUtils.SIZE_OF_BYTE); + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + } + + int valueLength = writable.getSize() - offsetTillNow; + if(getSaveKeys()) { + // Write ( key_length, value_length, key, + // value ) + valueStream.write(valueBytes, offsetTillNow, valueLength); + } else { + // Write (value_length + value) + valueStream.writeInt(valueLength); + valueStream.write(valueBytes, offsetTillNow, valueLength); + } + + numTuples++; + + // If we have multiple values for this md5 that is a collision, + // throw an exception--either the data itself has duplicates, there + // are trillions of keys, or someone is attempting something + // malicious ( We obviously expect collisions when we save keys ) + if(!getSaveKeys() && numTuples > 1) + throw new VoldemortException("Duplicate keys detected for md5 sum " + + ByteUtils.toHexString(ByteUtils.copy(key.get(), + 0, + key.getSize()))); + + } + + if(numTuples < 0) { + // Overflow + throw new VoldemortException("Found too many collisions: chunk " + chunkId + + " has exceeded " + Short.MAX_VALUE + " collisions."); + } else if(numTuples > 1) { + // Update number of collisions + max keys per collision + reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); + + long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); + if(numTuples > numCollisions) { + reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); + } + } + + // Flush the value + valueStream.flush(); + byte[] value = stream.toByteArray(); + + // Start writing to file now + // First, if save keys flag set the number of keys + if(getSaveKeys()) { + + this.valueFileStream.writeShort(numTuples); + this.position += ByteUtils.SIZE_OF_SHORT; + + if(this.checkSumDigestValue != null) { + this.checkSumDigestValue.update(numTuples); + } + } + + this.valueFileStream.write(value); + this.position += value.length; + + if(this.checkSumDigestValue != null) { + this.checkSumDigestValue.update(value); + } + + if(this.position < 0) + throw new VoldemortException("Chunk overflow exception: chunk " + chunkId + + " has exceeded " + Integer.MAX_VALUE + " bytes."); + } + + @Override + public void close() throws IOException { + + this.indexFileStream.close(); + this.valueFileStream.close(); + + if(this.nodeId == -1 || this.chunkId == -1 || this.partitionId == -1) { + // Issue 258 - No data was read in the reduce phase, do not create + // any output + return; + } + + // If the replica type read was not valid, shout out + if(getSaveKeys() && this.replicaType == -1) { + throw new RuntimeException("Could not read the replica type correctly for node " + + nodeId + " ( partition - " + this.partitionId + " )"); + } + + String fileNamePrefix = null; + if(getSaveKeys()) { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.replicaType) + "_" + + Integer.toString(this.chunkId)); + } else { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.chunkId)); + } + + // Initialize the node directory + Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); + + // Create output directory, if it doesn't exist + FileSystem outputFs = nodeDir.getFileSystem(this.conf); + outputFs.mkdirs(nodeDir); + + // Write the checksum and output files + if(this.checkSumType != CheckSumType.NONE) { + + if(this.checkSumDigestIndex != null && this.checkSumDigestValue != null) { + Path checkSumIndexFile = new Path(nodeDir, fileNamePrefix + ".index.checksum"); + Path checkSumValueFile = new Path(nodeDir, fileNamePrefix + ".data.checksum"); + + FSDataOutputStream output = outputFs.create(checkSumIndexFile); + output.write(this.checkSumDigestIndex.getCheckSum()); + output.close(); + + output = outputFs.create(checkSumValueFile); + output.write(this.checkSumDigestValue.getCheckSum()); + output.close(); + } else { + throw new RuntimeException("Failed to open checksum digest for node " + nodeId + + " ( partition - " + this.partitionId + ", chunk - " + + chunkId + " )"); + } + } + + // Generate the final chunk files + Path indexFile = new Path(nodeDir, fileNamePrefix + ".index"); + Path valueFile = new Path(nodeDir, fileNamePrefix + ".data"); + + logger.info("Moving " + this.taskIndexFileName + " to " + indexFile); + outputFs.rename(taskIndexFileName, indexFile); + logger.info("Moving " + this.taskValueFileName + " to " + valueFile); + outputFs.rename(this.taskValueFileName, valueFile); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java new file mode 100644 index 0000000000..3df92b85ba --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java @@ -0,0 +1,342 @@ +package voldemort.store.readonly.disk; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +public class HadoopStoreWriterPerBucket implements KeyValueWriter { + + private static final Logger logger = Logger.getLogger(HadoopStoreWriterPerBucket.class); + + private DataOutputStream[] indexFileStream = null; + private DataOutputStream[] valueFileStream = null; + private int[] position; + private String taskId = null; + + private int nodeId = -1; + private int partitionId = -1; + private int replicaType = -1; + + private Path[] taskIndexFileName; + private Path[] taskValueFileName; + + private JobConf conf; + private CheckSumType checkSumType; + private CheckSum[] checkSumDigestIndex; + private CheckSum[] checkSumDigestValue; + + private String outputDir; + + private FileSystem fs; + + @Override + public void conf(JobConf job) { + + JobConf conf = job; + try { + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + this.conf = job; + this.outputDir = job.get("final.output.dir"); + this.taskId = job.get("mapred.task.id"); + this.checkSumType = CheckSum.fromString(job.get("checksum.type")); + + this.checkSumDigestIndex = new CheckSum[getNumChunks()]; + this.checkSumDigestValue = new CheckSum[getNumChunks()]; + this.position = new int[getNumChunks()]; + this.taskIndexFileName = new Path[getNumChunks()]; + this.taskValueFileName = new Path[getNumChunks()]; + this.indexFileStream = new DataOutputStream[getNumChunks()]; + this.valueFileStream = new DataOutputStream[getNumChunks()]; + + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + + this.checkSumDigestIndex[chunkId] = CheckSum.getInstance(checkSumType); + this.checkSumDigestValue[chunkId] = CheckSum.getInstance(checkSumType); + this.position[chunkId] = 0; + + this.taskIndexFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), + getStoreName() + "." + + Integer.toString(chunkId) + + "_" + this.taskId + ".index"); + this.taskValueFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), + getStoreName() + "." + + Integer.toString(chunkId) + + "_" + this.taskId + ".data"); + + if(this.fs == null) + this.fs = this.taskIndexFileName[chunkId].getFileSystem(job); + + this.indexFileStream[chunkId] = fs.create(this.taskIndexFileName[chunkId]); + this.valueFileStream[chunkId] = fs.create(this.taskValueFileName[chunkId]); + + logger.info("Opening " + this.taskIndexFileName[chunkId] + " and " + + this.taskValueFileName[chunkId] + " for writing."); + } + + } catch(IOException e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + + } + + @Override + public void write(BytesWritable key, Iterator iterator, Reporter reporter) + throws IOException { + + // Read chunk id + int chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); + + // Write key and position + this.indexFileStream[chunkId].write(key.get(), 0, key.getSize()); + this.indexFileStream[chunkId].writeInt(this.position[chunkId]); + + // Run key through checksum digest + if(this.checkSumDigestIndex[chunkId] != null) { + this.checkSumDigestIndex[chunkId].update(key.get(), 0, key.getSize()); + this.checkSumDigestIndex[chunkId].update(this.position[chunkId]); + } + + short numTuples = 0; + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream valueStream = new DataOutputStream(stream); + + while(iterator.hasNext()) { + BytesWritable writable = iterator.next(); + byte[] valueBytes = writable.get(); + int offsetTillNow = 0; + + // Read node Id + if(this.nodeId == -1) + this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read partition id + if(this.partitionId == -1) + this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read replica type + if(getSaveKeys()) { + if(this.replicaType == -1) + this.replicaType = (int) ByteUtils.readBytes(valueBytes, + offsetTillNow, + ByteUtils.SIZE_OF_BYTE); + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + } + + int valueLength = writable.getSize() - offsetTillNow; + if(getSaveKeys()) { + // Write ( key_length, value_length, key, + // value ) + valueStream.write(valueBytes, offsetTillNow, valueLength); + } else { + // Write (value_length + value) + valueStream.writeInt(valueLength); + valueStream.write(valueBytes, offsetTillNow, valueLength); + } + + numTuples++; + + // If we have multiple values for this md5 that is a collision, + // throw an exception--either the data itself has duplicates, there + // are trillions of keys, or someone is attempting something + // malicious ( We obviously expect collisions when we save keys ) + if(!getSaveKeys() && numTuples > 1) + throw new VoldemortException("Duplicate keys detected for md5 sum " + + ByteUtils.toHexString(ByteUtils.copy(key.get(), + 0, + key.getSize()))); + + } + + if(numTuples < 0) { + // Overflow + throw new VoldemortException("Found too many collisions: chunk " + chunkId + + " has exceeded " + Short.MAX_VALUE + " collisions."); + } else if(numTuples > 1) { + // Update number of collisions + max keys per collision + reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); + + long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); + if(numTuples > numCollisions) { + reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); + } + } + + // Flush the value + valueStream.flush(); + byte[] value = stream.toByteArray(); + + // Start writing to file now + // First, if save keys flag set the number of keys + if(getSaveKeys()) { + + this.valueFileStream[chunkId].writeShort(numTuples); + this.position[chunkId] += ByteUtils.SIZE_OF_SHORT; + + if(this.checkSumDigestValue[chunkId] != null) { + this.checkSumDigestValue[chunkId].update(numTuples); + } + } + + this.valueFileStream[chunkId].write(value); + this.position[chunkId] += value.length; + + if(this.checkSumDigestValue[chunkId] != null) { + this.checkSumDigestValue[chunkId].update(value); + } + + if(this.position[chunkId] < 0) + throw new VoldemortException("Chunk overflow exception: chunk " + chunkId + + " has exceeded " + Integer.MAX_VALUE + " bytes."); + + } + + @Override + public void close() throws IOException { + + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + this.indexFileStream[chunkId].close(); + this.valueFileStream[chunkId].close(); + } + + if(this.nodeId == -1 || this.partitionId == -1) { + // Issue 258 - No data was read in the reduce phase, do not create + // any output + return; + } + + // If the replica type read was not valid, shout out + if(getSaveKeys() && this.replicaType == -1) { + throw new RuntimeException("Could not read the replica type correctly for node " + + nodeId + " ( partition - " + this.partitionId + " )"); + } + + String fileNamePrefix = null; + if(getSaveKeys()) { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.replicaType) + "_"); + } else { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_"); + } + + // Initialize the node directory + Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); + + // Create output directory, if it doesn't exist + FileSystem outputFs = nodeDir.getFileSystem(this.conf); + outputFs.mkdirs(nodeDir); + + // Write the checksum and output files + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + + String chunkFileName = fileNamePrefix + Integer.toString(chunkId); + if(this.checkSumType != CheckSumType.NONE) { + + if(this.checkSumDigestIndex[chunkId] != null + && this.checkSumDigestValue[chunkId] != null) { + Path checkSumIndexFile = new Path(nodeDir, chunkFileName + ".index.checksum"); + Path checkSumValueFile = new Path(nodeDir, chunkFileName + ".data.checksum"); + + FSDataOutputStream output = outputFs.create(checkSumIndexFile); + output.write(this.checkSumDigestIndex[chunkId].getCheckSum()); + output.close(); + + output = outputFs.create(checkSumValueFile); + output.write(this.checkSumDigestValue[chunkId].getCheckSum()); + output.close(); + } else { + throw new RuntimeException("Failed to open checksum digest for node " + nodeId + + " ( partition - " + this.partitionId + + ", chunk - " + chunkId + " )"); + } + } + + // Generate the final chunk files + Path indexFile = new Path(nodeDir, chunkFileName + ".index"); + Path valueFile = new Path(nodeDir, chunkFileName + ".data"); + + logger.info("Moving " + this.taskIndexFileName[chunkId] + " to " + indexFile); + fs.rename(taskIndexFileName[chunkId], indexFile); + logger.info("Moving " + this.taskValueFileName[chunkId] + " to " + valueFile); + fs.rename(this.taskValueFileName[chunkId], valueFile); + + } + + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java new file mode 100644 index 0000000000..ddc11c69e2 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java @@ -0,0 +1,23 @@ +package voldemort.store.readonly.disk; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; + +public interface KeyValueWriter { + + public static enum CollisionCounter { + + NUM_COLLISIONS, + MAX_COLLISIONS; + } + + public void conf(JobConf job); + + public void write(K key, Iterator iterator, Reporter reporter) throws IOException; + + public void close() throws IOException; + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java new file mode 100644 index 0000000000..05b14179e5 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java @@ -0,0 +1,296 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.io.StringReader; +import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.util.List; + +import org.apache.avro.generic.GenericData; +import org.apache.avro.mapred.AvroCollector; +import org.apache.avro.mapred.AvroMapper; +import org.apache.avro.mapred.Pair; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.routing.ConsistentRoutingStrategy; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.SerializerDefinition; +import voldemort.serialization.SerializerFactory; +import voldemort.serialization.avro.AvroGenericSerializer; +import voldemort.store.StoreDefinition; +import voldemort.store.compress.CompressionStrategy; +import voldemort.store.compress.CompressionStrategyFactory; +import voldemort.store.readonly.mr.azkaban.StoreBuilderTransformation; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; +import azkaban.common.utils.Props; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderMapper extends + AvroMapper> implements JobConfigurable { + + protected MessageDigest md5er; + protected ConsistentRoutingStrategy routingStrategy; + protected AvroGenericSerializer keySerializer; + protected AvroGenericSerializer valueSerializer; + + private String keySchema; + private String valSchema; + + private String keyField; + private String valField; + + private String _keySelection; + private String _valSelection; + + private StoreBuilderTransformation _keyTrans; + private StoreBuilderTransformation _valTrans; + + private CompressionStrategy valueCompressor; + private CompressionStrategy keyCompressor; + private SerializerDefinition keySerializerDefinition; + private SerializerDefinition valueSerializerDefinition; + + // Path path = new Path(fileName); + FSDataOutputStream outputStream; + + /** + * Create the voldemort key and value from the input key and value and map + * it out for each of the responsible voldemort nodes + * + * The output key is the md5 of the serialized key returned by makeKey(). + * The output value is the node_id & partition_id of the responsible node + * followed by serialized value returned by makeValue() OR if we have + * setKeys flag on the serialized key and serialized value + */ + @Override + public void map(GenericData.Record record, + AvroCollector> collector, + Reporter reporter) throws IOException { + + byte[] keyBytes = keySerializer.toBytes(record.get(keyField)); + byte[] valBytes = valueSerializer.toBytes(record.get(valField)); + + // Compress key and values if required + if(keySerializerDefinition.hasCompression()) { + keyBytes = keyCompressor.deflate(keyBytes); + } + + if(valueSerializerDefinition.hasCompression()) { + valBytes = valueCompressor.deflate(valBytes); + } + + // Get the output byte arrays ready to populate + byte[] outputValue; + BytesWritable outputKey; + + // Leave initial offset for (a) node id (b) partition id + // since they are written later + int offsetTillNow = 2 * ByteUtils.SIZE_OF_INT; + + if(getSaveKeys()) { + + // In order - 4 ( for node id ) + 4 ( partition id ) + 1 ( + // replica + // type - primary | secondary | tertiary... ] + 4 ( key size ) + // size ) + 4 ( value size ) + key + value + outputValue = new byte[valBytes.length + keyBytes.length + ByteUtils.SIZE_OF_BYTE + 4 + * ByteUtils.SIZE_OF_INT]; + + // Write key length - leave byte for replica type + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + ByteUtils.writeInt(outputValue, keyBytes.length, offsetTillNow); + + // Write value length + offsetTillNow += ByteUtils.SIZE_OF_INT; + ByteUtils.writeInt(outputValue, valBytes.length, offsetTillNow); + + // Write key + offsetTillNow += ByteUtils.SIZE_OF_INT; + System.arraycopy(keyBytes, 0, outputValue, offsetTillNow, keyBytes.length); + + // Write value + offsetTillNow += keyBytes.length; + System.arraycopy(valBytes, 0, outputValue, offsetTillNow, valBytes.length); + + // Generate MR key - upper 8 bytes of 16 byte md5 + outputKey = new BytesWritable(ByteUtils.copy(md5er.digest(keyBytes), + 0, + 2 * ByteUtils.SIZE_OF_INT)); + + } else { + + // In order - 4 ( for node id ) + 4 ( partition id ) + value + outputValue = new byte[valBytes.length + 2 * ByteUtils.SIZE_OF_INT]; + + // Write value + System.arraycopy(valBytes, 0, outputValue, offsetTillNow, valBytes.length); + + // Generate MR key - 16 byte md5 + outputKey = new BytesWritable(md5er.digest(keyBytes)); + + } + + // Generate partition and node list this key is destined for + List partitionList = routingStrategy.getPartitionList(keyBytes); + Node[] partitionToNode = routingStrategy.getPartitionToNode(); + + for(int replicaType = 0; replicaType < partitionList.size(); replicaType++) { + + // Node id + ByteUtils.writeInt(outputValue, + partitionToNode[partitionList.get(replicaType)].getId(), + 0); + + if(getSaveKeys()) { + // Primary partition id + ByteUtils.writeInt(outputValue, partitionList.get(0), ByteUtils.SIZE_OF_INT); + + // Replica type + ByteUtils.writeBytes(outputValue, + replicaType, + 2 * ByteUtils.SIZE_OF_INT, + ByteUtils.SIZE_OF_BYTE); + } else { + // Partition id + ByteUtils.writeInt(outputValue, + partitionList.get(replicaType), + ByteUtils.SIZE_OF_INT); + } + BytesWritable outputVal = new BytesWritable(outputValue); + + // System.out.println("collect length (K/V): "+ + // outputKey.getLength()+ " , " + outputVal.getLength()); + ByteBuffer keyBuffer = null, valueBuffer = null; + + byte[] md5KeyBytes = outputKey.getBytes(); + keyBuffer = ByteBuffer.allocate(md5KeyBytes.length); + keyBuffer.put(md5KeyBytes); + keyBuffer.rewind(); + + valueBuffer = ByteBuffer.allocate(outputValue.length); + valueBuffer.put(outputValue); + valueBuffer.rewind(); + + Pair p = new Pair(keyBuffer, + valueBuffer); + + collector.collect(p); + } + md5er.reset(); + } + + @Override + public void configure(JobConf conf) { + + super.setConf(conf); + // from parent code + + md5er = ByteUtils.getDigest("md5"); + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", true); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + + keySerializerDefinition = getStoreDef().getKeySerializer(); + valueSerializerDefinition = getStoreDef().getValueSerializer(); + + try { + SerializerFactory factory = new DefaultSerializerFactory(); + + if(conf.get("serializer.factory") != null) { + factory = (SerializerFactory) Class.forName(conf.get("serializer.factory")) + .newInstance(); + } + + keyField = conf.get("avro.key.field"); + valField = conf.get("avro.value.field"); + + keySchema = conf.get("avro.key.schema"); + valSchema = conf.get("avro.val.schema"); + + // hadoop.job.valueSchema + keySerializer = new AvroGenericSerializer(keySchema); + valueSerializer = new AvroGenericSerializer(valSchema); + } catch(Exception e) { + throw new RuntimeException(e); + } + + keyCompressor = new CompressionStrategyFactory().get(keySerializerDefinition.getCompression()); + valueCompressor = new CompressionStrategyFactory().get(valueSerializerDefinition.getCompression()); + + routingStrategy = new ConsistentRoutingStrategy(getCluster().getNodes(), + getStoreDef().getReplicationFactor()); + + // / + Props props = HadoopUtils.getPropsFromJob(conf); + + _keySelection = props.getString("key.selection", null); + _valSelection = props.getString("value.selection", null); + + String _keyTransClass = props.getString("key.transformation.class", null); + String _valueTransClass = props.getString("value.transformation.class", null); + + if(_keyTransClass != null) + _keyTrans = (StoreBuilderTransformation) Utils.callConstructor(_keyTransClass); + if(_valueTransClass != null) + _valTrans = (StoreBuilderTransformation) Utils.callConstructor(_valueTransClass); + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java new file mode 100644 index 0000000000..6c1c5a9dc2 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java @@ -0,0 +1,154 @@ +package voldemort.store.readonly.mr; + +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +import java.io.IOException; +import java.io.StringReader; +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Partitioner; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +/** + * A Partitioner that splits data so that all data for the same nodeId, chunkId + * combination ends up in the same reduce (and hence in the same store chunk) + */ +@SuppressWarnings("deprecation") +public class AvroStoreBuilderPartitioner implements + Partitioner, AvroValue> { + + @Override + public int getPartition(AvroKey key, AvroValue value, int numReduceTasks) { + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[key.datum().remaining()]; + key.datum().get(keyBytes); + + valueBytes = new byte[value.datum().remaining()]; + value.datum().get(valueBytes); + + BytesWritable outputKey = new BytesWritable(keyBytes); + BytesWritable outputVal = new BytesWritable(valueBytes); + + ByteBuffer keyBuffer = null, valueBuffer = null; + + keyBuffer = ByteBuffer.allocate(keyBytes.length); + keyBuffer.put(keyBytes); + keyBuffer.rewind(); + + valueBuffer = ByteBuffer.allocate(valueBytes.length); + valueBuffer.put(valueBytes); + valueBuffer.rewind(); + + key.datum(keyBuffer); + value.datum(valueBuffer); + + int partitionId = ByteUtils.readInt(valueBytes, ByteUtils.SIZE_OF_INT); + int chunkId = ReadOnlyUtils.chunk(keyBytes, getNumChunks()); + if(getSaveKeys()) { + int replicaType = (int) ByteUtils.readBytes(valueBytes, + 2 * ByteUtils.SIZE_OF_INT, + ByteUtils.SIZE_OF_BYTE); + if(getReducerPerBucket()) { + return (partitionId * getStoreDef().getReplicationFactor() + replicaType) + % numReduceTasks; + } else { + return ((partitionId * getStoreDef().getReplicationFactor() * getNumChunks()) + + (replicaType * getNumChunks()) + chunkId) + % numReduceTasks; + } + } else { + if(getReducerPerBucket()) { + return partitionId % numReduceTasks; + } else { + return (partitionId * getNumChunks() + chunkId) % numReduceTasks; + } + + } + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + @Override + public void configure(JobConf conf) { + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + } + + @SuppressWarnings("unused") + public void close() throws IOException {} + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java new file mode 100644 index 0000000000..6a8be8662b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java @@ -0,0 +1,92 @@ +package voldemort.store.readonly.mr; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.disk.HadoopStoreWriter; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderReducer implements + Reducer, AvroValue, Text, Text>, JobConfigurable, Closeable { + + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; + + @SuppressWarnings("unchecked") + @Override + public void reduce(AvroKey keyAvro, + Iterator> iterator, + OutputCollector collector, + Reporter reporter) throws IOException { + + ByteBuffer keyBuffer = keyAvro.datum(); + keyBuffer.rewind(); + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[keyBuffer.remaining()]; + keyBuffer.get(keyBytes); + + BytesWritable key = new BytesWritable(keyBytes); + + ArrayList valueList = new ArrayList(); + + while(iterator.hasNext()) { + ByteBuffer writable = iterator.next().datum(); + writable.rewind(); + // BytesWritable writable = iterator.next(); + valueBytes = null; + valueBytes = new byte[writable.remaining()]; + writable.get(valueBytes); + + BytesWritable value = new BytesWritable(valueBytes); + valueList.add(value); + + } + + writer.write(key, valueList.iterator(), reporter); + + } + + @Override + public void configure(JobConf job) { + + JobConf conf = job; + try { + + keyValueWriterClass = conf.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriter(); + + writer.conf(job); + + } catch(Exception e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + } + + @Override + public void close() throws IOException { + + writer.close(); + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java new file mode 100644 index 0000000000..d1f53d678b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java @@ -0,0 +1,94 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Closeable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.store.readonly.disk.HadoopStoreWriterPerBucket; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderReducerPerBucket implements + Reducer, AvroValue, Text, Text>, JobConfigurable, Closeable { + + private static final Logger logger = Logger.getLogger(AvroStoreBuilderReducerPerBucket.class); + + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; + + @Override + public void reduce(AvroKey keyAvro, + Iterator> iterator, + OutputCollector collector, + Reporter reporter) throws IOException { + + ByteBuffer keyBuffer = keyAvro.datum(); + keyBuffer.rewind(); + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[keyBuffer.remaining()]; + keyBuffer.get(keyBytes); + + BytesWritable key = new BytesWritable(keyBytes); + + ArrayList valueList = new ArrayList(); + + while(iterator.hasNext()) { + ByteBuffer writable = iterator.next().datum(); + writable.rewind(); + // BytesWritable writable = iterator.next(); + valueBytes = null; + valueBytes = new byte[writable.remaining()]; + writable.get(valueBytes); + + BytesWritable value = new BytesWritable(valueBytes); + valueList.add(value); + + } + + writer.write(key, valueList.iterator(), reporter); + + } + + @Override + public void configure(JobConf job) { + + JobConf conf = job; + try { + + keyValueWriterClass = conf.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriterPerBucket(); + + writer.conf(job); + + } catch(Exception e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + } + + @Override + public void close() throws IOException { + + writer.close(); + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java new file mode 100644 index 0000000000..3ea7783ca7 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java @@ -0,0 +1,20 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; + +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.mr.serialization.JsonMapper; + +public class IdentityJsonMapper extends JsonMapper { + + @Override + public void mapObjects(Object key, + Object value, + OutputCollector output, + Reporter reporter) throws IOException { + output.collect(key, value); + } + +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java new file mode 100644 index 0000000000..e4c7f3a21e --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java @@ -0,0 +1,22 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.mr.serialization.JsonReducer; + +public class IdentityJsonReducer extends JsonReducer { + + @Override + public void reduceObjects(Object key, + Iterator values, + OutputCollector collector, + Reporter reporter) throws IOException { + while(values.hasNext()) { + collector.collect(key, values.next()); + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java new file mode 100644 index 0000000000..d0473d4bfa --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java @@ -0,0 +1,73 @@ +package voldemort.store.readonly.mr; + +import java.util.Map; + +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.serialization.json.JsonTypeSerializer; +import voldemort.store.readonly.mr.azkaban.StoreBuilderTransformation; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.utils.Props; +import azkaban.common.utils.Utils; + +public class VoldemortStoreBuilderMapper extends AbstractHadoopStoreBuilderMapper { + + private String _keySelection; + private String _valSelection; + private JsonTypeSerializer _inputKeySerializer; + private JsonTypeSerializer _inputValueSerializer; + private StoreBuilderTransformation _keyTrans; + private StoreBuilderTransformation _valTrans; + + @Override + public Object makeKey(Object key, Object value) { + return makeResult((BytesWritable) key, _inputKeySerializer, _keySelection, _keyTrans); + } + + @Override + public Object makeValue(Object key, Object value) { + return makeResult((BytesWritable) value, _inputValueSerializer, _valSelection, _valTrans); + } + + private Object makeResult(BytesWritable writable, + JsonTypeSerializer serializer, + String selection, + StoreBuilderTransformation trans) { + Object obj = serializer.toObject(writable.get()); + if(selection != null) { + Map m = (Map) obj; + obj = m.get(selection); + } + + if(trans != null) + obj = trans.transform(obj); + + return obj; + } + + @Override + public void configure(JobConf conf) { + super.configure(conf); + Props props = HadoopUtils.getPropsFromJob(conf); + + _keySelection = props.getString("key.selection", null); + _valSelection = props.getString("value.selection", null); + _inputKeySerializer = getSchemaFromJob(conf, "mapper.input.key.schema"); + _inputValueSerializer = getSchemaFromJob(conf, "mapper.input.value.schema"); + String _keyTransClass = props.getString("key.transformation.class", null); + String _valueTransClass = props.getString("value.transformation.class", null); + + if(_keyTransClass != null) + _keyTrans = (StoreBuilderTransformation) Utils.callConstructor(_keyTransClass); + if(_valueTransClass != null) + _valTrans = (StoreBuilderTransformation) Utils.callConstructor(_valueTransClass); + } + + protected JsonTypeSerializer getSchemaFromJob(JobConf conf, String key) { + if(conf.get(key) == null) + throw new IllegalArgumentException("Missing required parameter '" + key + "' on job."); + return new JsonTypeSerializer(conf.get(key)); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java new file mode 100644 index 0000000000..afd8bc34e3 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java @@ -0,0 +1,268 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.mapred.Counters; +import org.apache.hadoop.mapred.Counters.Counter; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.RunningJob; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import voldemort.store.readonly.mr.IdentityJsonReducer; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +/** + * An abstract Base class for Hadoop Jobs + * + * @author bbansal + * + */ +public abstract class AbstractHadoopJob extends AbstractJob { + + public static String COMMON_FILE_DATE_PATTERN = "yyyy-MM-dd-HH-mm"; + public static final String HADOOP_PREFIX = "hadoop-conf."; + public static final String LATEST_SUFFIX = "#LATEST"; + public static final String CURRENT_SUFFIX = "#CURRENT"; + private final Props _props; + private RunningJob _runningJob; + + public AbstractHadoopJob(String name, Props props) { + super(name); + this._props = props; + } + + public void run(JobConf conf) throws Exception { + _runningJob = new JobClient(conf).submitJob(conf); + info("See " + _runningJob.getTrackingURL() + " for details."); + _runningJob.waitForCompletion(); + + if(!_runningJob.isSuccessful()) { + throw new Exception("Hadoop job:" + getId() + " failed!"); + } + + // dump all counters + Counters counters = _runningJob.getCounters(); + for(String groupName: counters.getGroupNames()) { + Counters.Group group = counters.getGroup(groupName); + info("Group: " + group.getDisplayName()); + for(Counter counter: group) + info(counter.getDisplayName() + ":\t" + counter.getValue()); + } + } + + public JobConf createJobConf(Class mapperClass) throws IOException, + URISyntaxException { + JobConf conf = createJobConf(mapperClass, IdentityJsonReducer.class); + conf.setNumReduceTasks(0); + + return conf; + } + + public JobConf createJobConf(Class mapperClass, + Class reducerClass, + Class combinerClass) throws IOException, + URISyntaxException { + JobConf conf = createJobConf(mapperClass, reducerClass); + conf.setCombinerClass(combinerClass); + + return conf; + } + + public JobConf createJobConf(Class mapperClass, + Class reducerClass) throws IOException, + URISyntaxException { + JobConf conf = new JobConf(); + // set custom class loader with custom find resource strategy. + + conf.setJobName(getId()); + conf.setMapperClass(mapperClass); + conf.setReducerClass(reducerClass); + + String hadoop_ugi = _props.getString("hadoop.job.ugi", null); + if(hadoop_ugi != null) { + conf.set("hadoop.job.ugi", hadoop_ugi); + } + + if(_props.getBoolean("is.local", false)) { + conf.set("mapred.job.tracker", "local"); + conf.set("fs.default.name", "file:///"); + conf.set("mapred.local.dir", "/tmp/map-red"); + + info("Running locally, no hadoop jar set."); + } else { + setClassLoaderAndJar(conf, getClass()); + info("Setting hadoop jar file for class:" + getClass() + " to " + conf.getJar()); + info("*************************************************************************"); + info(" Running on Real Hadoop Cluster(" + conf.get("mapred.job.tracker") + + ") "); + info("*************************************************************************"); + } + + // set JVM options if present + if(_props.containsKey("mapred.child.java.opts")) { + conf.set("mapred.child.java.opts", _props.getString("mapred.child.java.opts")); + info("mapred.child.java.opts set to " + _props.getString("mapred.child.java.opts")); + } + + // set input and output paths if they are present + if(_props.containsKey("input.paths")) { + List inputPaths = _props.getStringList("input.paths"); + if(inputPaths.size() == 0) + throw new IllegalArgumentException("Must specify at least one value for property 'input.paths'"); + for(String path: inputPaths) { + // Implied stuff, but good implied stuff + if(path.endsWith(LATEST_SUFFIX)) { + FileSystem fs = FileSystem.get(conf); + + PathFilter filter = new PathFilter() { + + @Override + public boolean accept(Path arg0) { + return !arg0.getName().startsWith("_") + && !arg0.getName().startsWith("."); + } + }; + + String latestPath = path.substring(0, path.length() - LATEST_SUFFIX.length()); + FileStatus[] statuses = fs.listStatus(new Path(latestPath), filter); + + Arrays.sort(statuses); + + path = statuses[statuses.length - 1].getPath().toString(); + System.out.println("Using latest folder: " + path); + } + HadoopUtils.addAllSubPaths(conf, new Path(path)); + } + } + + if(_props.containsKey("output.path")) { + String location = _props.get("output.path"); + if(location.endsWith("#CURRENT")) { + DateTimeFormatter format = DateTimeFormat.forPattern(COMMON_FILE_DATE_PATTERN); + String destPath = format.print(new DateTime()); + location = location.substring(0, location.length() - "#CURRENT".length()) + + destPath; + System.out.println("Store location set to " + location); + } + + FileOutputFormat.setOutputPath(conf, new Path(location)); + // For testing purpose only remove output file if exists + if(_props.getBoolean("force.output.overwrite", false)) { + FileSystem fs = FileOutputFormat.getOutputPath(conf).getFileSystem(conf); + fs.delete(FileOutputFormat.getOutputPath(conf), true); + } + } + + // Adds External jars to hadoop classpath + String externalJarList = _props.getString("hadoop.external.jarFiles", null); + if(externalJarList != null) { + String[] jarFiles = externalJarList.split(","); + for(String jarFile: jarFiles) { + info("Adding extenral jar File:" + jarFile); + DistributedCache.addFileToClassPath(new Path(jarFile), conf); + } + } + + // Adds distributed cache files + String cacheFileList = _props.getString("hadoop.cache.files", null); + if(cacheFileList != null) { + String[] cacheFiles = cacheFileList.split(","); + for(String cacheFile: cacheFiles) { + info("Adding Distributed Cache File:" + cacheFile); + DistributedCache.addCacheFile(new URI(cacheFile), conf); + } + } + + // Adds distributed cache files + String archiveFileList = _props.getString("hadoop.cache.archives", null); + if(archiveFileList != null) { + String[] archiveFiles = archiveFileList.split(","); + for(String archiveFile: archiveFiles) { + info("Adding Distributed Cache Archive File:" + archiveFile); + DistributedCache.addCacheArchive(new URI(archiveFile), conf); + } + } + + String hadoopCacheJarDir = _props.getString("hdfs.default.classpath.dir", null); + if(hadoopCacheJarDir != null) { + FileSystem fs = FileSystem.get(conf); + if(fs != null) { + FileStatus[] status = fs.listStatus(new Path(hadoopCacheJarDir)); + + if(status != null) { + for(int i = 0; i < status.length; ++i) { + if(!status[i].isDir()) { + Path path = new Path(hadoopCacheJarDir, status[i].getPath().getName()); + info("Adding Jar to Distributed Cache Archive File:" + path); + + DistributedCache.addFileToClassPath(path, conf); + } + } + } else { + info("hdfs.default.classpath.dir " + hadoopCacheJarDir + " is empty."); + } + } else { + info("hdfs.default.classpath.dir " + hadoopCacheJarDir + + " filesystem doesn't exist"); + } + } + + // May want to add this to HadoopUtils, but will await refactoring + for(String key: getProps().keySet()) { + String lowerCase = key.toLowerCase(); + if(lowerCase.startsWith(HADOOP_PREFIX)) { + String newKey = key.substring(HADOOP_PREFIX.length()); + conf.set(newKey, getProps().get(key)); + } + } + + HadoopUtils.setPropsInJob(conf, getProps()); + return conf; + } + + public Props getProps() { + return this._props; + } + + public void cancel() throws Exception { + if(_runningJob != null) + _runningJob.killJob(); + } + + public double getProgress() throws IOException { + if(_runningJob == null) + return 0.0; + else + return (double) (_runningJob.mapProgress() + _runningJob.reduceProgress()) / 2.0d; + } + + public Counters getCounters() throws IOException { + return _runningJob.getCounters(); + } + + public static void setClassLoaderAndJar(JobConf conf, Class jobClass) { + conf.setClassLoader(Thread.currentThread().getContextClassLoader()); + String jar = HadoopUtils.findContainingJar(jobClass, Thread.currentThread() + .getContextClassLoader()); + if(jar != null) { + conf.setJar(jar); + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java new file mode 100644 index 0000000000..3e6860ed04 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java @@ -0,0 +1,116 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +/** + * A test job that throws an exception + * + * @author bbansal Required Properties + *
    + *
  • voldemort.cluster.file
  • + *
  • voldemort.store.name
  • + *
  • input.path
  • + *
  • dest.path
  • + *
  • source.host
  • + *
  • dest.host
  • + *
+ */ +public abstract class AbstractVoldemortBatchCopyJob extends AbstractJob { + + private final Props _props; + + public AbstractVoldemortBatchCopyJob(String name, Props props) throws IOException { + super(name); + _props = props; + } + + public void run() throws Exception { + JobConf conf = new JobConf(); + HadoopUtils.copyInAllProps(_props, conf); + + Cluster cluster = HadoopUtils.readCluster(_props.get("voldemort.cluster.file"), conf); + final String storeName = _props.get("voldemort.store.name"); + final Path inputDir = new Path(_props.get("input.path")); + + ExecutorService executors = Executors.newFixedThreadPool(cluster.getNumberOfNodes()); + final Semaphore semaphore = new Semaphore(0, false); + final AtomicInteger countSuccess = new AtomicInteger(0); + final boolean[] succeeded = new boolean[cluster.getNumberOfNodes()]; + final String destinationDir = _props.get("dest.path"); + final String sourceHost = _props.getString("src.host", "localhost"); + + for(final Node node: cluster.getNodes()) { + + executors.execute(new Runnable() { + + public void run() { + int id = node.getId(); + String indexFile = inputDir + "/" + storeName + ".index" + "_" + + Integer.toString(id); + String dataFile = inputDir + "/" + storeName + ".data" + "_" + + Integer.toString(id); + + String host = node.getHost(); + try { + // copyFileToLocal(sourceHost, + // indexFile, + // host, + // VoldemortSwapperUtils.getIndexDestinationFile(node.getId(), + // destinationDir)); + // copyFileToLocal(sourceHost, + // dataFile, + // host, + // VoldemortSwapperUtils.getDataDestinationFile(node.getId(), + // destinationDir)); + + succeeded[node.getId()] = true; + countSuccess.incrementAndGet(); + } catch(Exception e) { + error("copy to Remote node failed for node:" + node.getId(), e); + } + + semaphore.release(); + } + }); + } + + // wait for all operations to complete + semaphore.acquire(cluster.getNumberOfNodes()); + + try { + if(countSuccess.get() == cluster.getNumberOfNodes() + || _props.getBoolean("swap.partial.index", false)) { + int counter = 0; + // lets try to swap only the successful nodes + for(Node node: cluster.getNodes()) { + // data refresh succeeded + if(succeeded[node.getId()]) { + VoldemortSwapperUtils.doSwap(storeName, node, destinationDir); + counter++; + } + } + info(counter + " node out of " + cluster.getNumberOfNodes() + + " refreshed with fresh index/data for store '" + storeName + "'"); + } else { + error("Failed to copy Index Files for the entire cluster."); + } + } finally { + // stop all executors Now + executors.shutdown(); + } + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java new file mode 100644 index 0000000000..91ba74370e --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java @@ -0,0 +1,59 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.util.Properties; + +/** + * This interface defines a Raw Job interface. Each job defines + *
    + *
  • Job Type : {HADOOP, UNIX, JAVA, SUCCESS_TEST, CONTROLLER}
  • + *
  • Job ID/Name : {String}
  • + *
  • Arguments: Key/Value Map for Strings
  • + *
+ * + * A job is required to have a constructor Job(String jobId, Props props) + */ + +public interface Job { + + /** + * Returns a unique(should be checked in xml) string name/id for the Job. + * + * @return + */ + public String getId(); + + /** + * Run the job. In general this method can only be run once. Must either + * succeed or throw an exception. + */ + public void run() throws Exception; + + /** + * Best effort attempt to cancel the job. + * + * @throws Exception If cancel fails + */ + public void cancel() throws Exception; + + /** + * Returns a progress report between [0 - 1.0] to indicate the percentage + * complete + * + * @throws Exception If getting progress fails + */ + public double getProgress() throws Exception; + + /** + * Get the generated properties from this job. + * + * @return + */ + public Properties getJobGeneratedProperties(); + + /** + * Determine if the job was cancelled. + * + * @return + */ + public boolean isCanceled(); +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java new file mode 100644 index 0000000000..6703f6879a --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java @@ -0,0 +1,14 @@ +package voldemort.store.readonly.mr.azkaban; + +/** + * An interface to use for processing rows in the voldemort store builder + * + * @author jkreps + * + */ +public interface StoreBuilderTransformation +{ + + public Object transform(Object obj); + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java new file mode 100644 index 0000000000..cbfe09504b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java @@ -0,0 +1,11 @@ +package voldemort.store.readonly.mr.azkaban; + +public class UndefinedPropertyException extends RuntimeException { + + private static final long serialVersionUID = 1; + + public UndefinedPropertyException(String message) { + super(message); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java new file mode 100644 index 0000000000..cbe1d28f78 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java @@ -0,0 +1,405 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.mapred.lib.HashPartitioner; +import org.apache.log4j.Logger; + +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.routing.ConsistentRoutingStrategy; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.Serializer; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.mr.serialization.JsonConfigurable; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.utils.ByteUtils; +import azkaban.common.utils.Props; + +/** + * Creates Index and value files using Voldemort hash keys for easy batch + * update. + *

+ * Creates two files + *

    + *
  • Index File: Keeps the position Index for each key sorted by MD5(key) + * chunk-size is KEY_HASH_SIZE(16 bytes) + POSITION_SIZE(8 bytes)
  • + *
  • Values file: saves the value corrosponding to the Key MD5
  • + *
      + *

      + * Required Properties + *

        + *
      • voldemort.cluster.file
      • + *
      • voldemort.storedef.file
      • + *
      • voldemort.store.name
      • + *
      • voldemort.store.version
      • + *
      • input.data.check.percent
      • + *
      + * + * @author bbansal + * + * @deprecated Use {@link VoldemortStoreBuilderJob} instead. + * + */ +@Deprecated +public class VoldemortBatchIndexJob extends AbstractHadoopJob { + + private Cluster _cluster = null; + + private static Logger logger = Logger.getLogger(VoldemortStoreBuilderJob.class); + + public VoldemortBatchIndexJob(String name, Props props) throws FileNotFoundException { + super(name, props); + } + + /** + * @deprecated use + * {@link VoldemortStoreBuilderJob#execute(String, String, String, String, int)} + * the parameter voldemort store version is deprecated and no + * longer used. Version is read from the store definition + * istead. + *

      + * @param voldemortClusterLocalFile + * @param storeName + * @param inputPath + * @param outputPath + * @param voldemortStoreVersion + * @param voldemortCheckDataPercent + * @throws IOException + * @throws URISyntaxException + */ + @Deprecated + public void execute(String voldemortClusterLocalFile, + String storeName, + String inputPath, + String outputPath, + int voldemortStoreVersion, + int voldemortCheckDataPercent) throws IOException, URISyntaxException { + execute(voldemortClusterLocalFile, + storeName, + inputPath, + outputPath, + voldemortCheckDataPercent); + } + + /** + * Method to allow this process to be a instance call from another Job. + * + * @storeName to dump the value + * @inputFile to generate the VFILE + * + * + */ + public void execute(String voldemortClusterLocalFile, + String storeName, + String inputPath, + String outputPath, + int voldemortCheckDataPercent) throws IOException, URISyntaxException { + JobConf conf = createJobConf(VoldemortBatchIndexMapper.class, + VoldemortBatchIndexReducer.class); + + try { + // get the voldemort cluster definition + // We need to use cluster.xml here where it not yet localized by + // TaskRunner + _cluster = HadoopUtils.readCluster(voldemortClusterLocalFile, conf); + } catch(Exception e) { + logger.error("Failed to read Voldemort cluster details", e); + throw new RuntimeException("", e); + } + + // set the partitioner + conf.setPartitionerClass(VoldemortBatchIndexPartitoner.class); + conf.setNumReduceTasks(_cluster.getNumberOfNodes()); + + // Blow Away the O/p if force.overwirte is available + + FileInputFormat.setInputPaths(conf, inputPath); + + FileOutputFormat.setOutputPath(conf, new Path(outputPath)); + + if(getProps().getBoolean("force.output.overwrite", false)) { + FileSystem fs = FileOutputFormat.getOutputPath(conf).getFileSystem(conf); + fs.delete(FileOutputFormat.getOutputPath(conf), true); + } + + conf.setInputFormat(SequenceFileInputFormat.class); + conf.setOutputFormat(SequenceFileOutputFormat.class); + conf.setMapOutputKeyClass(BytesWritable.class); + conf.setMapOutputValueClass(BytesWritable.class); + conf.setOutputKeyClass(BytesWritable.class); + conf.setOutputValueClass(BytesWritable.class); + + conf.setNumReduceTasks(_cluster.getNumberOfNodes()); + + // get the store information + + conf.setStrings("voldemort.index.filename", storeName + ".index"); + conf.setStrings("voldemort.data.filename", storeName + ".data"); + conf.setInt("input.data.check.percent", voldemortCheckDataPercent); + conf.setStrings("voldemort.store.name", storeName); + + // run(conf); + JobClient.runJob(conf); + + } + + @Override + public void run() throws Exception { + + execute(getProps().get("voldemort.cluster.file"), + getProps().get("voldemort.store.name"), + getProps().get("input.path"), + getProps().get("output.path"), + getProps().getInt("input.data.check.percent", 0)); + + } + + /** + * TODO HIGH : Doesnot check with Voldemort schema should validate the + * voldemort schema before writing. + * + * @author bbansal + * + */ + public static class VoldemortBatchIndexMapper extends JsonConfigurable implements + Mapper { + + private static Logger logger = Logger.getLogger(VoldemortBatchIndexMapper.class); + private Cluster _cluster = null; + private StoreDefinition _storeDef = null; + private ConsistentRoutingStrategy _routingStrategy = null; + private Serializer _keySerializer; + private Serializer _valueSerializer; + private int _checkPercent; + private int _version; + + public void map(BytesWritable key, + BytesWritable value, + OutputCollector output, + Reporter reporter) throws IOException { + byte[] keyBytes = ByteUtils.copy(key.get(), 0, key.getSize()); + byte[] valBytes = ByteUtils.copy(value.get(), 0, value.getSize()); + + ByteArrayOutputStream versionedKey = new ByteArrayOutputStream(); + DataOutputStream keyDin = new DataOutputStream(versionedKey); + keyDin.write(_version); + keyDin.write(keyBytes); + keyDin.close(); + + if(logger.isDebugEnabled()) { + logger.debug("Original key: size:" + versionedKey.toByteArray().length + " val:" + + ByteUtils.toHexString(versionedKey.toByteArray())); + logger.debug("MD5 val: size:" + ByteUtils.md5(versionedKey.toByteArray()).length + + " val:" + + ByteUtils.toHexString(ByteUtils.md5(versionedKey.toByteArray()))); + logger.debug(" value bytes:" + value.getSize() + " [" + + ByteUtils.toHexString(valBytes) + "]"); + } + + List nodes = _routingStrategy.routeRequest(keyBytes); + for(Node node: nodes) { + ByteArrayOutputStream versionedValue = new ByteArrayOutputStream(); + DataOutputStream valueDin = new DataOutputStream(versionedValue); + valueDin.writeInt(node.getId()); + valueDin.write(_version); + valueDin.write(valBytes); + valueDin.close(); + + // check input + if(Math.ceil(Math.random() * 100.0) < _checkPercent) { + checkJsonType(versionedKey.toByteArray(), + ByteUtils.copy(versionedValue.toByteArray(), + 4, + versionedValue.size())); + } + + BytesWritable outputKey = new BytesWritable(ByteUtils.md5(versionedKey.toByteArray())); + BytesWritable outputVal = new BytesWritable(versionedValue.toByteArray()); + + output.collect(outputKey, outputVal); + } + } + + public void checkJsonType(byte[] key, byte[] value) { + try { + _keySerializer.toObject(key); + _valueSerializer.toObject(value); + } catch(Exception e) { + throw new RuntimeException("Failed to Serialize key/Value check data and config schema.", + e); + } + } + + public void configure(JobConf conf) { + Props props = HadoopUtils.getPropsFromJob(conf); + + // get the voldemort cluster.xml and store.xml files. + try { + _cluster = HadoopUtils.readCluster(props.get("voldemort.cluster.file"), conf); + _storeDef = HadoopUtils.readStoreDef(props.get("voldemort.store.file"), + props.get("voldemort.store.name"), + conf); + + _checkPercent = conf.getInt("input.data.check.percent", 0); + _routingStrategy = new ConsistentRoutingStrategy(_cluster.getNodes(), + _storeDef.getReplicationFactor()); + _keySerializer = (Serializer) new DefaultSerializerFactory().getSerializer(_storeDef.getKeySerializer()); + _valueSerializer = (Serializer) new DefaultSerializerFactory().getSerializer(_storeDef.getValueSerializer()); + + _version = _storeDef.getKeySerializer().getCurrentSchemaVersion(); + _routingStrategy = new ConsistentRoutingStrategy(_cluster.getNodes(), + _storeDef.getReplicationFactor()); + + if(_routingStrategy == null) { + throw new RuntimeException("Failed to create routing strategy"); + } + } catch(Exception e) { + logger.error("Failed to read Voldemort cluster/storeDef details", e); + throw new RuntimeException("", e); + } + } + } + + public static class VoldemortBatchIndexPartitoner extends + HashPartitioner { + + @Override + public int getPartition(BytesWritable key, BytesWritable value, int numReduceTasks) { + // The partition id is first 4 bytes in the value. + DataInputStream buffer = new DataInputStream(new ByteArrayInputStream(value.get())); + int nodeId = -2; + try { + nodeId = buffer.readInt(); + } catch(IOException e) { + throw new RuntimeException("Failed to parse nodeId from buffer.", e); + } + return (nodeId) % numReduceTasks; + } + } + + public static class VoldemortBatchIndexReducer implements + Reducer { + + private DataOutputStream _indexFileStream = null; + private DataOutputStream _valueFileStream = null; + + private long _position = 0; + + private JobConf _conf = null; + private String _taskId = "dummy"; + private int _nodeId = -1; + + String indexFileName; + String dataFileName; + Path taskIndexFileName; + Path taskValueFileName; + String storeName; + + /** + * Reduce should get sorted MD5 keys here with a single value (appended + * in begining with 4 bits of nodeId) + */ + public void reduce(BytesWritable key, + Iterator values, + OutputCollector output, + Reporter reporter) throws IOException { + byte[] keyBytes = ByteUtils.copy(key.get(), 0, key.getSize()); + + while(values.hasNext()) { + BytesWritable value = values.next(); + byte[] valBytes = ByteUtils.copy(value.get(), 0, value.getSize()); + + if(_nodeId == -1) { + DataInputStream buffer = new DataInputStream(new ByteArrayInputStream(valBytes)); + _nodeId = buffer.readInt(); + } + // strip first 4 bytes from value here added in mapper for + // partitioner + // convenience. + byte[] value1 = ByteUtils.copy(valBytes, 4, valBytes.length); + + if(logger.isDebugEnabled()) { + logger.debug("Reduce Original key: size:" + keyBytes.length + " val:" + + ByteUtils.toHexString(keyBytes)); + logger.debug("Reduce value bytes:" + value1.length + " [" + + ByteUtils.toHexString(value1) + "]"); + } + + // Write Index Key/ position + _indexFileStream.write(keyBytes); + _indexFileStream.writeLong(_position); + + _valueFileStream.writeInt(value1.length); + _valueFileStream.write(value1); + _position += value1.length + 4; + + if(_position < 0) { + logger.error("Position bigger than Integer size, split input files."); + System.exit(1); + } + } + + } + + public void configure(JobConf job) { + Props props = HadoopUtils.getPropsFromJob(job); + + try { + _position = 0; + _conf = job; + + _taskId = job.get("mapred.task.id"); + + storeName = props.get("voldemort.store.name"); + taskIndexFileName = new Path(FileOutputFormat.getOutputPath(_conf), + _conf.get("voldemort.index.filename") + "_" + _taskId); + taskValueFileName = new Path(FileOutputFormat.getOutputPath(_conf), + _conf.get("voldemort.data.filename") + "_" + _taskId); + + FileSystem fs = taskIndexFileName.getFileSystem(job); + + _indexFileStream = fs.create(taskIndexFileName, (short) 1); + _valueFileStream = fs.create(taskValueFileName, (short) 1); + } catch(IOException e) { + throw new RuntimeException("Failed to open Input/OutputStream", e); + } + } + + public void close() throws IOException { + + _indexFileStream.close(); + _valueFileStream.close(); + + Path hdfsIndexFile = new Path(FileOutputFormat.getOutputPath(_conf), _nodeId + ".index"); + Path hdfsValueFile = new Path(FileOutputFormat.getOutputPath(_conf), _nodeId + ".data"); + + FileSystem fs = hdfsIndexFile.getFileSystem(_conf); + fs.rename(taskIndexFileName, hdfsIndexFile); + fs.rename(taskValueFileName, hdfsValueFile); + } + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBuildAndPushJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBuildAndPushJob.java new file mode 100644 index 0000000000..dfb7392283 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBuildAndPushJob.java @@ -0,0 +1,792 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.log4j.Logger; + +import voldemort.client.protocol.admin.AdminClient; +import voldemort.client.protocol.admin.AdminClientConfig; +import voldemort.cluster.Cluster; +import voldemort.serialization.SerializerDefinition; +import voldemort.serialization.json.JsonTypeDefinition; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.store.readonly.mr.azkaban.VoldemortStoreBuilderJob.VoldemortStoreBuilderConf; +import voldemort.store.readonly.mr.azkaban.VoldemortSwapJob.VoldemortSwapConf; +import voldemort.store.readonly.mr.utils.AvroUtils; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.store.readonly.mr.utils.JsonSchema; +import voldemort.store.readonly.mr.utils.VoldemortUtils; +import voldemort.utils.Utils; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +public class VoldemortBuildAndPushJob extends AbstractJob { + + private final Logger log; + + private final Props props; + + private Cluster cluster; + + private List storeDefs; + + private final String storeName; + + private final List clusterUrl; + + private final int nodeId; + + private final List dataDirs; + + private final boolean isAvroJob; + + private final String keyField; + + private final String valueField; + + /* Informed stuff */ + private final String informedURL = "http://informed.corp.linkedin.com/_post"; + private final List informedResults; + private ExecutorService informedExecutor; + + public VoldemortBuildAndPushJob(String name, Props props) { + super(name); + this.props = props; + this.storeName = props.getString("push.store.name").trim(); + this.clusterUrl = new ArrayList(); + this.dataDirs = new ArrayList(); + + String clusterUrlText = props.getString("push.cluster"); + for(String url: Utils.COMMA_SEP.split(clusterUrlText.trim())) + if(url.trim().length() > 0) + this.clusterUrl.add(url); + + if(clusterUrl.size() <= 0) + throw new RuntimeException("Number of urls should be atleast 1"); + + // Support multiple output dirs if the user mentions only "push", no + // "build". + // If user mentions both then should have only one + String dataDirText = props.getString("build.output.dir"); + for(String dataDir: Utils.COMMA_SEP.split(dataDirText.trim())) + if(dataDir.trim().length() > 0) + this.dataDirs.add(dataDir); + + if(dataDirs.size() <= 0) + throw new RuntimeException("Number of data dirs should be atleast 1"); + + this.nodeId = props.getInt("push.node", 0); + this.log = Logger.getLogger(name); + this.informedResults = Lists.newArrayList(); + this.informedExecutor = Executors.newFixedThreadPool(2); + + isAvroJob = props.getBoolean("build.type.avro", false); + + keyField = props.getString("avro.key.field", "key"); + + valueField = props.getString("avro.value.field", "value"); + + System.out.println("keyfield = " + keyField); + log.info("valueField = " + valueField); + } + + @Override + public void run() throws Exception { + boolean build = props.getBoolean("build", true); + boolean push = props.getBoolean("push", true); + + if(build && push && dataDirs.size() != 1) { + // Should have only one data directory ( which acts like the parent + // directory to all + // urls ) + throw new RuntimeException(" Should have only one data directory ( which acts like root directory ) since they are auto-generated during build phase "); + } else if(!build && push && dataDirs.size() != clusterUrl.size()) { + // Number of data directories should be equal to number of cluster + // urls + throw new RuntimeException(" Since we are only pushing, number of data directories ( comma separated ) should be equal to number of cluster urls "); + } + + // Check every url individually + HashMap exceptions = Maps.newHashMap(); + + for(int index = 0; index < clusterUrl.size(); index++) { + String url = clusterUrl.get(index); + + log.info("Working on " + url); + + try { + if(isAvroJob) + verifyAvroSchema(url); + else + verifySchema(url); + + String buildOutputDir; + if(build) { + buildOutputDir = runBuildStore(props, url); + } else { + buildOutputDir = dataDirs.get(index); + } + + if(push) { + if(log.isDebugEnabled()) + log.debug("Informing about push start ..."); + informedResults.add(this.informedExecutor.submit(new InformedClient(this.props, + "Running", + this.getId()))); + + runPushStore(props, url, buildOutputDir); + } + + if(build && push && !props.getBoolean("build.output.keep", false)) { + JobConf jobConf = new JobConf(); + + if(props.containsKey("hadoop.job.ugi")) { + jobConf.set("hadoop.job.ugi", props.getString("hadoop.job.ugi")); + } + + log.info("Deleting " + buildOutputDir); + HadoopUtils.deletePathIfExists(jobConf, buildOutputDir); + log.info("Deleted " + buildOutputDir); + } + + if(log.isDebugEnabled()) + log.debug("Informing about push finish ..."); + informedResults.add(this.informedExecutor.submit(new InformedClient(this.props, + "Finished", + this.getId()))); + + for(Future result: informedResults) { + try { + result.get(); + } catch(Exception e) { + this.log.error("Exception in consumer", e); + } + } + this.informedExecutor.shutdownNow(); + } catch(Exception e) { + log.error("Exception during build and push for url " + url, e); + exceptions.put(url, e); + } + } + + if(exceptions.size() > 0) { + log.error("Got exceptions while pushing to " + Joiner.on(",").join(exceptions.keySet()) + + " => " + Joiner.on(",").join(exceptions.values())); + System.exit(-1); + } + } + + public void verifySchema(String url) throws Exception { + // create new json store def with schema from the metadata in the input + // path + JsonSchema schema = HadoopUtils.getSchemaFromPath(getInputPath()); + int replicationFactor = props.getInt("build.replication.factor", 2); + int requiredReads = props.getInt("build.required.reads", 1); + int requiredWrites = props.getInt("build.required.writes", 1); + String description = props.getString("push.store.description", ""); + String owners = props.getString("push.store.owners", ""); + String keySchema = "\n\t\tjson\n\t\t" + + schema.getKeyType() + "\n\t"; + String valSchema = "\n\t\tjson\n\t\t" + + schema.getValueType() + "\n\t"; + + boolean hasCompression = false; + if(props.containsKey("build.compress.value")) + hasCompression = true; + + if(hasCompression) { + valSchema += "\tgzip\n\t"; + } + + if(props.containsKey("build.force.schema.key")) { + keySchema = props.get("build.force.schema.key"); + } + + if(props.containsKey("build.force.schema.value")) { + valSchema = props.get("build.force.schema.value"); + } + + String newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + (props.containsKey("push.force.schema.key")) ? props.getString("push.force.schema.key") + : keySchema, + (props.containsKey("push.force.schema.value")) ? props.getString("push.force.schema.value") + : valSchema, + description, + owners); + + log.info("Verifying store: \n" + newStoreDefXml.toString()); + + StoreDefinition newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + // get store def from cluster + log.info("Getting store definition from: " + url + " (node id " + this.nodeId + ")"); + + AdminClient adminClient = new AdminClient(url, new AdminClientConfig()); + try { + List remoteStoreDefs = adminClient.getRemoteStoreDefList(this.nodeId) + .getValue(); + boolean foundStore = false; + + // go over all store defs and see if one has the same name as the + // store we're trying + // to build + for(StoreDefinition remoteStoreDef: remoteStoreDefs) { + if(remoteStoreDef.getName().equals(storeName)) { + // if the store already exists, but doesn't match what we + // want to push, we need + // to worry + if(!remoteStoreDef.equals(newStoreDef)) { + // it is possible that the stores actually DO match, but + // the + // json in the key/value serializers is out of order (eg + // {'a': 'int32', 'b': 'int32'} could have a/b reversed. + // this is just a reflection of the fact that voldemort + // json + // type defs use hashmaps that are unordered, and pig + // uses + // bags that are unordered as well. it's therefore + // unpredictable what order the keys will come out of + // pig. + // let's check to see if the key/value serializers are + // REALLY equal. + SerializerDefinition localKeySerializerDef = newStoreDef.getKeySerializer(); + SerializerDefinition localValueSerializerDef = newStoreDef.getValueSerializer(); + SerializerDefinition remoteKeySerializerDef = remoteStoreDef.getKeySerializer(); + SerializerDefinition remoteValueSerializerDef = remoteStoreDef.getValueSerializer(); + + if(remoteKeySerializerDef.getName().equals("json") + && remoteValueSerializerDef.getName().equals("json") + && remoteKeySerializerDef.getAllSchemaInfoVersions().size() == 1 + && remoteValueSerializerDef.getAllSchemaInfoVersions().size() == 1) { + JsonTypeDefinition remoteKeyDef = JsonTypeDefinition.fromJson(remoteKeySerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition remoteValDef = JsonTypeDefinition.fromJson(remoteValueSerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition localKeyDef = JsonTypeDefinition.fromJson(localKeySerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition localValDef = JsonTypeDefinition.fromJson(localValueSerializerDef.getCurrentSchemaInfo()); + + if(remoteKeyDef.equals(localKeyDef) && remoteValDef.equals(localValDef)) { + String compressionPolicy = ""; + if(hasCompression) { + compressionPolicy = "\n\t\tgzip"; + } + + // if the key/value serializers are REALLY equal + // (even though the strings may not match), then + // just use the remote stores to GUARANTEE that + // they + // match, and try again. + newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + "\n\t\tjson\n\t\t" + + remoteKeySerializerDef.getCurrentSchemaInfo() + + "\n\t", + "\n\t\tjson\n\t\t" + + remoteValueSerializerDef.getCurrentSchemaInfo() + + "" + + compressionPolicy + + "\n\t"); + + newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + if(!remoteStoreDef.equals(newStoreDef)) { + // if we still get a fail, then we know that + // the + // store defs don't match for reasons OTHER + // than + // the key/value serializer + throw new RuntimeException("Your store schema is identical, but the store definition does not match. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } else { + // if the key/value serializers are not equal + // (even + // in java, not just json strings), then fail + throw new RuntimeException("Your store definition does not match the store definition that is already in the cluster. Tried to resolve identical schemas between local and remote, but failed. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } + } + + foundStore = true; + break; + } + } + + // if the store doesn't exist yet, create it + if(!foundStore) { + // New requirement - Make sure the user had description and + // owner specified + if(description.length() == 0) { + throw new RuntimeException("Description field missing in store definition. " + + "Please add \"push.store.description\" with a line describing your store"); + } + + if(owners.length() == 0) { + throw new RuntimeException("Owner field missing in store definition. " + + "Please add \"push.store.owners\" with value being comma-separated list of LinkedIn email ids"); + + } + + log.info("Could not find store " + storeName + + " on Voldemort. Adding it to all nodes "); + adminClient.addStore(newStoreDef); + } + + // don't use newStoreDef because we want to ALWAYS use the JSON + // definition since the store builder assumes that you are using + // JsonTypeSerializer. This allows you to tweak your value/key store + // xml + // as you see fit, but still uses the json sequence file meta data + // to + // build the store. + storeDefs = ImmutableList.of(VoldemortUtils.getStoreDef(VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + keySchema, + valSchema))); + cluster = adminClient.getAdminClientCluster(); + } finally { + adminClient.stop(); + } + } + + public String runBuildStore(Props props, String url) throws Exception { + int replicationFactor = props.getInt("build.replication.factor", 2); + int chunkSize = props.getInt("build.chunk.size", 1024 * 1024 * 1024); + Path tempDir = new Path(props.getString("build.temp.dir", "/tmp/vold-build-and-push-" + + new Random().nextLong())); + URI uri = new URI(url); + Path outputDir = new Path(props.getString("build.output.dir"), uri.getHost()); + Path inputPath = getInputPath(); + String keySelection = props.getString("build.key.selection", null); + String valSelection = props.getString("build.value.selection", null); + CheckSumType checkSumType = CheckSum.fromString(props.getString("checksum.type", + CheckSum.toString(CheckSumType.MD5))); + boolean saveKeys = props.getBoolean("save.keys", true); + boolean reducerPerBucket = props.getBoolean("reducer.per.bucket", false); + int numChunks = props.getInt("num.chunks", -1); + + if(isAvroJob) { + String recSchema = getRecordSchema(); + String keySchema = getKeySchema(); + String valSchema = getValueSchema(); + + new VoldemortStoreBuilderJob(this.getId() + "-build-store", + props, + new VoldemortStoreBuilderConf(replicationFactor, + chunkSize, + tempDir, + outputDir, + inputPath, + cluster, + storeDefs, + storeName, + keySelection, + valSelection, + null, + null, + checkSumType, + saveKeys, + reducerPerBucket, + numChunks, + keyField, + valueField, + recSchema, + keySchema, + valSchema), true).run(); + return outputDir.toString(); + } + new VoldemortStoreBuilderJob(this.getId() + "-build-store", + props, + new VoldemortStoreBuilderConf(replicationFactor, + chunkSize, + tempDir, + outputDir, + inputPath, + cluster, + storeDefs, + storeName, + keySelection, + valSelection, + null, + null, + checkSumType, + saveKeys, + reducerPerBucket, + numChunks)).run(); + return outputDir.toString(); + } + + public void runPushStore(Props props, String url, String dataDir) throws Exception { + // For backwards compatibility http timeout = admin timeout + int httpTimeoutMs = 1000 * props.getInt("push.http.timeout.seconds", 24 * 60 * 60); + long pushVersion = props.getLong("push.version", -1L); + if(props.containsKey("push.version.timestamp")) { + DateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); + pushVersion = Long.parseLong(format.format(new Date())); + } + int maxBackoffDelayMs = 1000 * props.getInt("push.backoff.delay.seconds", 60); + boolean rollback = props.getBoolean("push.rollback", true); + + new VoldemortSwapJob(this.getId() + "-push-store", + props, + new VoldemortSwapConf(cluster, + dataDir, + storeName, + httpTimeoutMs, + pushVersion, + maxBackoffDelayMs, + rollback)).run(); + } + + /** + * Get the sanitized input path. At the moment of writing, this means the + * #LATEST tag is expanded. + */ + private Path getInputPath() throws IOException { + Path path = new Path(props.getString("build.input.path")); + return HadoopUtils.getSanitizedPath(path); + } + + public String getRecordSchema() throws IOException { + Schema schema = AvroUtils.getAvroSchemaFromPath(getInputPath()); + + // schema.getField("key").schema(); + String recSchema = schema.toString(); + + return recSchema; + + } + + public String getKeySchema() throws IOException { + Schema schema = AvroUtils.getAvroSchemaFromPath(getInputPath()); + + // schema.getField("key").schema(); + String keySchema = schema.getField(keyField).schema().toString(); + + return keySchema; + + } + + public String getValueSchema() throws IOException { + Schema schema = AvroUtils.getAvroSchemaFromPath(getInputPath()); + + // schema.getField("key").schema(); + String valueSchema = schema.getField(valueField).schema().toString(); + + return valueSchema; + + } + + public void verifyAvroSchema(String url) throws Exception { + // create new n store def with schema from the metadata in the input + // path + Schema schema = AvroUtils.getAvroSchemaFromPath(getInputPath()); + int replicationFactor = props.getInt("build.replication.factor", 2); + int requiredReads = props.getInt("build.required.reads", 1); + int requiredWrites = props.getInt("build.required.writes", 1); + String description = props.getString("push.store.description", ""); + String owners = props.getString("push.store.owners", ""); + // String keySchema = schema.getField("key").schema(); + + // schema.getField("key").schema(); + String keySchema = "\n\t\tavro-generic\n\t\t" + + schema.getField(keyField).schema() + "\n\t"; + String valSchema = "\n\t\tavro-generic\n\t\t" + + schema.getField(valueField).schema() + "\n\t"; + + boolean hasCompression = false; + if(props.containsKey("build.compress.value")) + hasCompression = true; + + if(hasCompression) { + valSchema += "\tgzip\n\t"; + } + + if(props.containsKey("build.force.schema.key")) { + keySchema = props.get("build.force.schema.key"); + } + + if(props.containsKey("build.force.schema.value")) { + valSchema = props.get("build.force.schema.value"); + } + + String newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + (props.containsKey("push.force.schema.key")) ? props.getString("push.force.schema.key") + : keySchema, + (props.containsKey("push.force.schema.value")) ? props.getString("push.force.schema.value") + : valSchema, + description, + owners); + + log.info("Verifying store: \n" + newStoreDefXml.toString()); + + StoreDefinition newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + // get store def from cluster + log.info("Getting store definition from: " + url + " (node id " + this.nodeId + ")"); + + AdminClient adminClient = new AdminClient(url, new AdminClientConfig()); + try { + List remoteStoreDefs = adminClient.getRemoteStoreDefList(this.nodeId) + .getValue(); + boolean foundStore = false; + + // go over all store defs and see if one has the same name as the + // store we're trying + // to build + for(StoreDefinition remoteStoreDef: remoteStoreDefs) { + if(remoteStoreDef.getName().equals(storeName)) { + // if the store already exists, but doesn't match what we + // want to push, we need + // to worry + if(!remoteStoreDef.equals(newStoreDef)) { + // it is possible that the stores actually DO match, but + // the + // json in the key/value serializers is out of order (eg + // {'a': 'int32', 'b': 'int32'} could have a/b reversed. + // this is just a reflection of the fact that voldemort + // json + // type defs use hashmaps that are unordered, and pig + // uses + // bags that are unordered as well. it's therefore + // unpredictable what order the keys will come out of + // pig. + // let's check to see if the key/value serializers are + // REALLY equal. + SerializerDefinition localKeySerializerDef = newStoreDef.getKeySerializer(); + SerializerDefinition localValueSerializerDef = newStoreDef.getValueSerializer(); + SerializerDefinition remoteKeySerializerDef = remoteStoreDef.getKeySerializer(); + SerializerDefinition remoteValueSerializerDef = remoteStoreDef.getValueSerializer(); + + if(remoteKeySerializerDef.getName().equals("avro-generic") + && remoteValueSerializerDef.getName().equals("avro-generic") + && remoteKeySerializerDef.getAllSchemaInfoVersions().size() == 1 + && remoteValueSerializerDef.getAllSchemaInfoVersions().size() == 1) { + Schema remoteKeyDef = Schema.parse(remoteKeySerializerDef.getCurrentSchemaInfo()); + Schema remoteValDef = Schema.parse(remoteValueSerializerDef.getCurrentSchemaInfo()); + Schema localKeyDef = Schema.parse(localKeySerializerDef.getCurrentSchemaInfo()); + Schema localValDef = Schema.parse(localValueSerializerDef.getCurrentSchemaInfo()); + + if(remoteKeyDef.equals(localKeyDef) && remoteValDef.equals(localValDef)) { + String compressionPolicy = ""; + if(hasCompression) { + compressionPolicy = "\n\t\tgzip"; + } + + // if the key/value serializers are REALLY equal + // (even though the strings may not match), then + // just use the remote stores to GUARANTEE that + // they + // match, and try again. + newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + "\n\t\tavro-generic\n\t\t" + + remoteKeySerializerDef.getCurrentSchemaInfo() + + "\n\t", + "\n\t\tavro-generic\n\t\t" + + remoteValueSerializerDef.getCurrentSchemaInfo() + + "" + + compressionPolicy + + "\n\t"); + + newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + if(!remoteStoreDef.equals(newStoreDef)) { + // if we still get a fail, then we know that + // the + // store defs don't match for reasons OTHER + // than + // the key/value serializer + throw new RuntimeException("Your store schema is identical, but the store definition does not match. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } else { + // if the key/value serializers are not equal + // (even + // in java, not just json strings), then fail + throw new RuntimeException("Your store definition does not match the store definition that is already in the cluster. Tried to resolve identical schemas between local and remote, but failed. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } + } + + foundStore = true; + break; + } + } + + // if the store doesn't exist yet, create it + if(!foundStore) { + // New requirement - Make sure the user had description and + // owner specified + if(description.length() == 0) { + throw new RuntimeException("Description field missing in store definition. " + + "Please add \"push.store.description\" with a line describing your store"); + } + + if(owners.length() == 0) { + throw new RuntimeException("Owner field missing in store definition. " + + "Please add \"push.store.owners\" with value being comma-separated list of LinkedIn email ids"); + + } + + log.info("Could not find store " + storeName + + " on Voldemort. Adding it to all nodes "); + adminClient.addStore(newStoreDef); + } + + storeDefs = ImmutableList.of(VoldemortUtils.getStoreDef(VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + props.containsKey("build.preferred.reads") ? props.getInt("build.preferred.reads") + : null, + props.containsKey("build.preferred.writes") ? props.getInt("build.preferred.writes") + : null, + keySchema, + valSchema))); + cluster = adminClient.getAdminClientCluster(); + } finally { + adminClient.stop(); + } + } + + private class InformedClient implements Runnable { + + private Props props; + private String status; + private String source; + + public InformedClient(Props props, String status, String source) { + this.props = props; + this.status = status; + this.source = source; + } + + @SuppressWarnings("unchecked") + @Override + public void run() { + try { + URL url = new URL(informedURL); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod("POST"); + conn.setDoOutput(true); + conn.setDoInput(true); + conn.setRequestProperty("Content-Type", "application/json"); + + String storeName = this.props.getString("push.store.name", "null"); + String clusterName = this.props.getString("push.cluster", "null"); + String owners = this.props.getString("push.store.owners", "null"); + String replicationFactor = this.props.getString("build.replication.factor", "null"); + + // JSON Object did not work for some reason. Hence doing my own + // Json. + String message = "Store : " + storeName.replaceAll("[\'\"]", "") + ", Status : " + + this.status.replaceAll("[\'\"]", "") + ", URL : " + + clusterName.replaceAll("[\'\"]", "") + ", owners : " + + owners.replaceAll("[\'\"]", "") + ", replication : " + + replicationFactor.replaceAll("[\'\"]", ""); + String payloadStr = "{\"message\":\"" + message + + "\",\"topic\":\"build-and-push\",\"source\":\"" + this.source + + "\",\"user\":\"bandp\"}"; + if(log.isDebugEnabled()) + log.debug("Payload : " + payloadStr); + + OutputStream out = conn.getOutputStream(); + out.write(payloadStr.getBytes()); + out.close(); + + if(conn.getResponseCode() != 200) { + System.out.println(conn.getResponseCode()); + log.error("Illegal response : " + conn.getResponseMessage()); + throw new IOException(conn.getResponseMessage()); + } + + // Buffer the result into a string + BufferedReader rd = new BufferedReader(new InputStreamReader(conn.getInputStream())); + StringBuilder sb = new StringBuilder(); + String line; + while((line = rd.readLine()) != null) { + sb.append(line); + } + rd.close(); + + if(log.isDebugEnabled()) + log.debug("Received response: " + sb); + + conn.disconnect(); + + } catch(Exception e) { + log.error(e.getMessage()); + e.printStackTrace(); + } + } + + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortMultiStoreBuildAndPushJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortMultiStoreBuildAndPushJob.java new file mode 100644 index 0000000000..b4f573881d --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortMultiStoreBuildAndPushJob.java @@ -0,0 +1,834 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.client.protocol.admin.AdminClientConfig; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.serialization.SerializerDefinition; +import voldemort.serialization.json.JsonTypeDefinition; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.store.readonly.mr.azkaban.VoldemortStoreBuilderJob.VoldemortStoreBuilderConf; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.store.readonly.mr.utils.JsonSchema; +import voldemort.store.readonly.mr.utils.VoldemortUtils; +import voldemort.store.readonly.swapper.AdminStoreSwapper; +import voldemort.utils.Pair; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; + +public class VoldemortMultiStoreBuildAndPushJob extends AbstractJob { + + private final Logger log; + + private final Props props; + + private List storeNames; + + private final List clusterUrls; + + /** + * The input directories to use on a per store name basis + */ + private final HashMap inputDirsPerStore; + + /** + * The final output that the output directory is stored in is as follows - [ + * outputDir ]/[ store_name ]/[ cluster_name ] + */ + private final Path outputDir; + + /** + * The node id to which we'll query and check if the stores already exists. + * If the store doesn't exist, it creates it. + */ + private final int nodeId; + + /** + * Get the sanitized input path. At the moment of writing, this means the + * #LATEST tag is expanded. + */ + private Path getPath(String pathString) throws IOException { + Path path = new Path(pathString); + return HadoopUtils.getSanitizedPath(path); + } + + public VoldemortMultiStoreBuildAndPushJob(String name, Props props) throws IOException { + super(name); + this.props = props; + this.log = Logger.getLogger(name); + this.nodeId = props.getInt("check.node", 0); + + // Get the input directories + List inputDirsPathString = VoldemortUtils.getCommaSeparatedStringValues(props.getString("build.input.path"), + "input directory"); + + // Get the store names + this.storeNames = VoldemortUtils.getCommaSeparatedStringValues(props.getString("push.store.name"), + "store name"); + + // Check if the number of stores = input directories ( obviously ) + if(this.storeNames.size() != inputDirsPathString.size()) { + throw new RuntimeException("Number of stores ( " + this.storeNames.size() + + " ) is not equal to number of input directories ( " + + inputDirsPathString.size() + " )"); + } + + // Convert them to Path + this.inputDirsPerStore = Maps.newHashMap(); + int index = 0; + for(String inputDirPathString: inputDirsPathString) { + this.inputDirsPerStore.put(storeNames.get(index), getPath(inputDirPathString)); + index++; + } + + // Get the output directory + String outputDirString = props.getString("build.output.dir", + "/tmp/voldemort-build-and-push-temp-" + + new Random().nextLong()); + this.outputDir = getPath(outputDirString); + + log.info("Storing output of all push jobs in " + this.outputDir); + + // Get the cluster urls to push to + this.clusterUrls = VoldemortUtils.getCommaSeparatedStringValues(props.getString("push.cluster"), + "cluster urls"); + + } + + /** + * Given the filesystem and a path recursively goes in and calculates the + * size + * + * @param fs Filesystem + * @param path The root path whose length we need to calculate + * @return The length in long + * @throws IOException + */ + public long sizeOfPath(FileSystem fs, Path path) throws IOException { + long size = 0; + FileStatus[] statuses = fs.listStatus(path); + if(statuses != null) { + for(FileStatus status: statuses) { + if(status.isDir()) + size += sizeOfPath(fs, status.getPath()); + else + size += status.getLen(); + } + } + return size; + } + + @Override + public void run() throws Exception { + + // Mapping of Pair [ cluster url, store name ] to List of previous node + // directories. + // Required for rollback... + Multimap, Pair> previousNodeDirPerClusterStore = HashMultimap.create(); + + // Retrieve filesystem information for checking if folder exists + final FileSystem fs = outputDir.getFileSystem(new Configuration()); + + // Step 1 ) Order the stores depending on the size of the store + TreeMap storeNameSortedBySize = Maps.newTreeMap(); + for(String storeName: storeNames) { + storeNameSortedBySize.put(sizeOfPath(fs, inputDirsPerStore.get(storeName)), storeName); + } + + log.info("Store names along with their input file sizes - " + storeNameSortedBySize); + + // This will collect it in ascending order of size + this.storeNames = Lists.newArrayList(storeNameSortedBySize.values()); + + // Reverse it such that is in descending order of size + Collections.reverse(this.storeNames); + + log.info("Store names in the order of which we'll run build and push - " + this.storeNames); + + // Step 2 ) Get the push version if set + final long pushVersion = props.containsKey("push.version.timestamp") ? Long.parseLong(new SimpleDateFormat("yyyyMMddHHmmss").format(new Date())) + : props.getLong("push.version", + -1L); + + // Mapping of Pair [ cluster url, store name ] to Future with list of + // node dirs + HashMap, Future>> fetchDirsPerStoreCluster = Maps.newHashMap(); + + // Store mapping of url to cluster metadata + final ConcurrentHashMap urlToCluster = new ConcurrentHashMap(); + + // Mapping of Pair [ cluster url, store name ] to List of node + // directories + final HashMap, List> nodeDirPerClusterStore = new HashMap, List>(); + + // Iterate over all of them and check if they are complete + final HashMap, Exception> exceptions = Maps.newHashMap(); + + ExecutorService executor = null; + try { + executor = Executors.newFixedThreadPool(props.getInt("build.push.parallel", 1)); + + // Step 3 ) Start the building + pushing of all stores in parallel + for(final String storeName: storeNames) { + // Go over every cluster and do the build phase + for(int index = 0; index < clusterUrls.size(); index++) { + final String url = clusterUrls.get(index); + fetchDirsPerStoreCluster.put(Pair.create(url, storeName), + executor.submit(new Callable>() { + + @Override + public List call() throws Exception { + + log.info("========= Working on build + push phase for store '" + + storeName + + "' and cluster '" + + url + "' =========="); + + // Create an admin + // client which will be + // used by + // everyone + AdminClient adminClient = null; + + // Executor inside + // executor - your mind + // just + // exploded! + ExecutorService internalExecutor = null; + + try { + // Retrieve admin + // client for + // verification of + // schema + pushing + adminClient = new AdminClient(url, + new AdminClientConfig()); + + // Verify the store + // exists ( If not, + // add it + // the + // store ) + Pair metadata = verifySchema(storeName, + url, + inputDirsPerStore.get(storeName), + adminClient); + + // Populate the url + // to cluster + // metadata + urlToCluster.put(url, + metadata.getSecond()); + + // Create output + // directory path + URI uri = new URI(url); + + Path outputDirPath = new Path(outputDir + + Path.SEPARATOR + + storeName, + uri.getHost()); + + log.info("Running build phase for store '" + + storeName + + "' and url '" + + url + + "'. Reading from input directory '" + + inputDirsPerStore.get(storeName) + + "' and writing to " + + outputDirPath); + + runBuildStore(metadata.getSecond(), + metadata.getFirst(), + inputDirsPerStore.get(storeName), + outputDirPath); + + log.info("Finished running build phase for store " + + storeName + + " and url '" + + url + + "'. Written to directory " + + outputDirPath); + + long storePushVersion = pushVersion; + if(storePushVersion == -1L) { + log.info("Retrieving version number for store '" + + storeName + + "' and cluster '" + + url + + "'"); + + Map pushVersions = adminClient.getROMaxVersion(Lists.newArrayList(storeName)); + + if(pushVersions == null + || !pushVersions.containsKey(storeName)) { + throw new RuntimeException("Could not retrieve version for store '" + + storeName + + "'"); + } + + storePushVersion = pushVersions.get(storeName); + storePushVersion++; + + log.info("Retrieved max version number for store '" + + storeName + + "' and cluster '" + + url + + "' = " + + storePushVersion); + } + + log.info("Running push for cluster url " + + url); + + // Used for + // parallel pushing + internalExecutor = Executors.newCachedThreadPool(); + + AdminStoreSwapper swapper = new AdminStoreSwapper(metadata.getSecond(), + internalExecutor, + adminClient, + 1000 * props.getInt("timeout.seconds", + 24 * 60 * 60), + true, + true); + + // Convert to + // hadoop specific + // path + String outputDirPathString = outputDirPath.makeQualified(fs) + .toString(); + + if(!fs.exists(outputDirPath)) { + throw new RuntimeException("Output directory for store " + + storeName + + " and cluster '" + + url + + "' - " + + outputDirPathString + + " does not exist"); + } + + log.info("Pushing data to store '" + + storeName + "' on cluster " + + url + " from path " + + outputDirPathString + + " with version " + + storePushVersion); + + List nodeDirs = swapper.invokeFetch(storeName, + outputDirPathString, + storePushVersion); + + log.info("Successfully pushed data to store '" + + storeName + + "' on cluster " + + url + + " from path " + + outputDirPathString + + " with version " + + storePushVersion); + + return nodeDirs; + } finally { + if(internalExecutor != null) { + internalExecutor.shutdownNow(); + internalExecutor.awaitTermination(10, + TimeUnit.SECONDS); + } + if(adminClient != null) { + adminClient.stop(); + } + } + } + + })); + + } + + } + + for(final String storeName: storeNames) { + for(int index = 0; index < clusterUrls.size(); index++) { + Pair key = Pair.create(clusterUrls.get(index), storeName); + Future> nodeDirs = fetchDirsPerStoreCluster.get(key); + try { + nodeDirPerClusterStore.put(key, nodeDirs.get()); + } catch(Exception e) { + exceptions.put(key, e); + } + } + } + + } finally { + if(executor != null) { + executor.shutdownNow(); + executor.awaitTermination(10, TimeUnit.SECONDS); + } + } + + // ===== If we got exceptions during the build + push, delete data from + // successful + // nodes ====== + if(!exceptions.isEmpty()) { + + log.error("Got an exception during pushes. Deleting data already pushed on successful nodes"); + + for(int index = 0; index < clusterUrls.size(); index++) { + String clusterUrl = clusterUrls.get(index); + Cluster cluster = urlToCluster.get(clusterUrl); + + AdminClient adminClient = null; + try { + adminClient = new AdminClient(cluster, new AdminClientConfig()); + for(final String storeName: storeNames) { + // Check if the [ cluster , store name ] succeeded. We + // need to roll it back + Pair key = Pair.create(clusterUrl, storeName); + + if(nodeDirPerClusterStore.containsKey(key)) { + List nodeDirs = nodeDirPerClusterStore.get(key); + + log.info("Deleting data for successful pushes to " + clusterUrl + + " and store " + storeName); + int nodeId = 0; + for(String nodeDir: nodeDirs) { + try { + log.info("Deleting data ( " + nodeDir + + " ) for successful pushes to '" + clusterUrl + + "' and store '" + storeName + "' and node " + nodeId); + adminClient.failedFetchStore(nodeId, storeName, nodeDir); + log.info("Successfully deleted data for successful pushes to '" + + clusterUrl + "' and store '" + storeName + + "' and node " + nodeId); + + } catch(Exception e) { + log.error("Failure while deleting data on node " + nodeId + + " for store '" + storeName + "' and url '" + + clusterUrl + "'"); + } + nodeId++; + } + } + } + } finally { + if(adminClient != null) { + adminClient.stop(); + } + } + } + + int errorNo = 1; + for(Pair key: exceptions.keySet()) { + log.error("Error no " + errorNo + "] Error pushing for cluster '" + key.getFirst() + + "' and store '" + key.getSecond() + "' :", exceptions.get(key)); + errorNo++; + } + + throw new VoldemortException("Exception during build + push"); + } + + // ====== Delete the temporary directory since we don't require it + // ====== + if(!props.getBoolean("build.output.keep", false)) { + JobConf jobConf = new JobConf(); + + if(props.containsKey("hadoop.job.ugi")) { + jobConf.set("hadoop.job.ugi", props.getString("hadoop.job.ugi")); + } + + log.info("Deleting output directory since we have finished the pushes " + outputDir); + HadoopUtils.deletePathIfExists(jobConf, outputDir.toString()); + log.info("Successfully deleted output directory since we have finished the pushes" + + outputDir); + } + + // ====== Time to swap the stores one node at a time ======== + try { + for(int index = 0; index < clusterUrls.size(); index++) { + String url = clusterUrls.get(index); + Cluster cluster = urlToCluster.get(url); + + AdminClient adminClient = new AdminClient(cluster, new AdminClientConfig()); + + log.info("Swapping all stores on cluster " + url); + try { + // Go over every node and swap + for(Node node: cluster.getNodes()) { + + log.info("Swapping all stores on cluster " + url + " and node " + + node.getId()); + + // Go over every store and swap + for(String storeName: storeNames) { + + Pair key = Pair.create(url, storeName); + log.info("Swapping '" + storeName + "' store on cluster " + url + + " and node " + node.getId() + " - " + + nodeDirPerClusterStore.get(key).get(node.getId())); + + previousNodeDirPerClusterStore.put(key, + Pair.create(node.getId(), + adminClient.swapStore(node.getId(), + storeName, + nodeDirPerClusterStore.get(key) + .get(node.getId())))); + log.info("Successfully swapped '" + storeName + "' store on cluster " + + url + " and node " + node.getId()); + + } + + } + } finally { + if(adminClient != null) { + adminClient.stop(); + } + } + } + } catch(Exception e) { + + log.error("Got an exception during swaps. Rolling back data already pushed on successful nodes"); + + for(Pair clusterStoreTuple: previousNodeDirPerClusterStore.keySet()) { + Collection> nodeToPreviousDirs = previousNodeDirPerClusterStore.get(clusterStoreTuple); + String url = clusterStoreTuple.getFirst(); + Cluster cluster = urlToCluster.get(url); + + log.info("Rolling back for cluster " + url + " and store " + + clusterStoreTuple.getSecond()); + + AdminClient adminClient = new AdminClient(cluster, new AdminClientConfig()); + try { + for(Pair nodeToPreviousDir: nodeToPreviousDirs) { + log.info("Rolling back for cluster " + url + " and store " + + clusterStoreTuple.getSecond() + " and node " + + nodeToPreviousDir.getFirst() + " to dir " + + nodeToPreviousDir.getSecond()); + adminClient.rollbackStore(nodeToPreviousDir.getFirst(), + nodeToPreviousDir.getSecond(), + ReadOnlyUtils.getVersionId(new File(nodeToPreviousDir.getSecond()))); + log.info("Successfully rolled back for cluster " + url + " and store " + + clusterStoreTuple.getSecond() + " and node " + + nodeToPreviousDir.getFirst() + " to dir " + + nodeToPreviousDir.getSecond()); + + } + } finally { + if(adminClient != null) { + adminClient.stop(); + } + } + } + throw e; + } + } + + /** + * Verify if the store exists on the cluster ( pointed by url ). Also use + * the input path to retrieve the metadata + * + * @param storeName Store name + * @param url The url of the cluster + * @param inputPath The input path where the files exist. This will be used + * for building the store + * @param adminClient Admin Client used to verify the schema + * @return Returns a pair of store definition + cluster metadata + * @throws IOException Exception due to input path being bad + */ + public Pair verifySchema(String storeName, + String url, + Path inputPath, + AdminClient adminClient) throws IOException { + // create new json store def with schema from the metadata in the input + // path + JsonSchema schema = HadoopUtils.getSchemaFromPath(inputPath); + int replicationFactor = props.getInt("build.replication.factor." + storeName, + props.getInt("build.replication.factor", 2)); + int requiredReads = props.getInt("build.required.reads." + storeName, + props.getInt("build.required.reads", 1)); + int requiredWrites = props.getInt("build.required.writes." + storeName, + props.getInt("build.required.writes", 1)); + + int preferredReads = props.getInt("build.preferred.reads." + storeName, + props.getInt("build.preferred.reads", -1)); + int preferredWrites = props.getInt("build.preferred.writes." + storeName, + props.getInt("build.preferred.writes", -1)); + + String description = props.getString("push.store.description." + storeName, + props.getString("push.store.description", "")); + String owners = props.getString("push.store.owners." + storeName, + props.getString("push.store.owners", "")); + + // Generate the key and value schema + String keySchema = "\n\t\tjson\n\t\t" + + schema.getKeyType() + "\n\t"; + String valSchema = "\n\t\tjson\n\t\t" + + schema.getValueType() + "\n\t"; + + String keySchemaCompression = ""; + if(props.containsKey("build.compress.key." + storeName) + || (storeNames.size() == 1 && props.containsKey("build.compress.key"))) { + keySchemaCompression = "\tgzip\n\t"; + keySchema += keySchemaCompression; + } + + String valueSchemaCompression = ""; + if(props.containsKey("build.compress.value." + storeName) + || (storeNames.size() == 1 && props.containsKey("build.compress.value"))) { + valueSchemaCompression = "\tgzip\n\t"; + valSchema += valueSchemaCompression; + } + + if(props.containsKey("build.force.schema.key." + storeName)) { + keySchema = props.get("build.force.schema.key." + storeName); + } + + if(props.containsKey("build.force.schema.value." + storeName)) { + valSchema = props.get("build.force.schema.value." + storeName); + } + + // For backwards compatibility check build.force.schema.* + if(props.containsKey("build.force.schema.key") && storeNames.size() == 1) { + keySchema = props.get("build.force.schema.key"); + } + + if(props.containsKey("build.force.schema.value") && storeNames.size() == 1) { + valSchema = props.get("build.force.schema.value"); + } + + String newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + (preferredReads < 0) ? null + : preferredReads, + (preferredWrites < 0) ? null + : preferredWrites, + keySchema, + valSchema, + description, + owners); + + log.info("Verifying store: \n" + newStoreDefXml.toString()); + + StoreDefinition newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + // get store def from cluster + log.info("Getting store definition from: " + url + " ( node id " + this.nodeId + " )"); + + List remoteStoreDefs = adminClient.getRemoteStoreDefList(this.nodeId) + .getValue(); + boolean foundStore = false; + + // go over all store defs and see if one has the same name as the store + // we're trying + // to build + for(StoreDefinition remoteStoreDef: remoteStoreDefs) { + if(remoteStoreDef.getName().equals(storeName)) { + // if the store already exists, but doesn't match what we want + // to push, we need + // to worry + if(!remoteStoreDef.equals(newStoreDef)) { + // it is possible that the stores actually DO match, but the + // json in the key/value serializers is out of order (eg + // {'a': 'int32', 'b': 'int32'} could have a/b reversed. + // this is just a reflection of the fact that voldemort json + // type defs use hashmaps that are unordered, and pig uses + // bags that are unordered as well. it's therefore + // unpredictable what order the keys will come out of pig. + // let's check to see if the key/value serializers are + // REALLY equal. + SerializerDefinition localKeySerializerDef = newStoreDef.getKeySerializer(); + SerializerDefinition localValueSerializerDef = newStoreDef.getValueSerializer(); + SerializerDefinition remoteKeySerializerDef = remoteStoreDef.getKeySerializer(); + SerializerDefinition remoteValueSerializerDef = remoteStoreDef.getValueSerializer(); + + if(remoteKeySerializerDef.getName().equals("json") + && remoteValueSerializerDef.getName().equals("json") + && remoteKeySerializerDef.getAllSchemaInfoVersions().size() == 1 + && remoteValueSerializerDef.getAllSchemaInfoVersions().size() == 1) { + JsonTypeDefinition remoteKeyDef = JsonTypeDefinition.fromJson(remoteKeySerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition remoteValDef = JsonTypeDefinition.fromJson(remoteValueSerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition localKeyDef = JsonTypeDefinition.fromJson(localKeySerializerDef.getCurrentSchemaInfo()); + JsonTypeDefinition localValDef = JsonTypeDefinition.fromJson(localValueSerializerDef.getCurrentSchemaInfo()); + + if(remoteKeyDef.equals(localKeyDef) && remoteValDef.equals(localValDef)) { + // if the key/value serializers are REALLY equal + // (even though the strings may not match), then + // just use the remote stores to GUARANTEE that they + // match, and try again. + keySchema = "\n\t\tjson\n\t\t" + + remoteKeySerializerDef.getCurrentSchemaInfo() + + "\n\t" + keySchemaCompression; + valSchema = "\n\t\tjson\n\t\t" + + remoteValueSerializerDef.getCurrentSchemaInfo() + + "\n\t" + valueSchemaCompression; + newStoreDefXml = VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + (preferredReads < 0) ? null + : preferredReads, + (preferredWrites < 0) ? null + : preferredWrites, + keySchema, + valSchema, + description, + owners); + + newStoreDef = VoldemortUtils.getStoreDef(newStoreDefXml); + + if(!remoteStoreDef.equals(newStoreDef)) { + // if we still get a fail, then we know that the + // store defs don't match for reasons OTHER than + // the key/value serializer + throw new RuntimeException("Your store schema is identical, but the store definition does not match. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } else { + // if the key/value serializers are not equal (even + // in java, not just json strings), then fail + throw new RuntimeException("Your store definition does not match the store definition that is already in the cluster. Tried to resolve identical schemas between local and remote, but failed. Have: " + + newStoreDef + + "\nBut expected: " + + remoteStoreDef); + } + } + } + + foundStore = true; + break; + } + } + + // if the store doesn't exist yet, create it + if(!foundStore) { + // New requirement - Make sure the user had description and owner + // specified + if(description.length() == 0) { + throw new RuntimeException("Description field missing in store definition. " + + "Please add \"push.store.description\" with a line describing your store"); + } + + if(owners.length() == 0) { + throw new RuntimeException("Owner field missing in store definition. " + + "Please add \"push.store.owners\" with value being comma-separated list of LinkedIn email ids"); + + } + + log.info("Could not find store " + storeName + + " on Voldemort. Adding it to all nodes for cluster " + url); + adminClient.addStore(newStoreDef); + } + + // don't use newStoreDef because we want to ALWAYS use the JSON + // definition since the store builder assumes that you are using + // JsonTypeSerializer. This allows you to tweak your value/key store xml + // as you see fit, but still uses the json sequence file meta data to + // build the store. + StoreDefinition storeDef = VoldemortUtils.getStoreDef(VoldemortUtils.getStoreDefXml(storeName, + replicationFactor, + requiredReads, + requiredWrites, + (preferredReads < 0) ? null + : preferredReads, + (preferredWrites < 0) ? null + : preferredWrites, + keySchema, + valSchema, + description, + owners)); + Cluster cluster = adminClient.getAdminClientCluster(); + + return Pair.create(storeDef, cluster); + } + + /** + * Only build the store + * + * @param cluster Cluster metadata + * @param storeDef Store definition metadata + * @param inputPath The input path where the current data is present + * @param outputPath The output location where we'll like to store our data + * @throws Exception + */ + public void runBuildStore(Cluster cluster, + StoreDefinition storeDef, + Path inputPath, + Path outputPath) throws Exception { + int chunkSize = props.getInt("build.chunk.size." + storeDef.getName(), + props.getInt("build.chunk.size", (int) 1024 * 1024 * 1024)); + Path tempDir = new Path(props.getString("build.temp.dir", "/tmp/voldemort-build-temp-" + + new Random().nextLong())); + + String keySelection = props.getString("build.key.selection." + storeDef.getName(), + props.getString("build.key.selection", null)); + String valSelection = props.getString("build.value.selection." + storeDef.getName(), + props.getString("build.key.selection", null)); + + int numChunks = props.getInt("num.chunks." + storeDef.getName(), + props.getInt("num.chunks", -1)); + + CheckSumType checkSumType = CheckSum.fromString(props.getString("checksum.type", + CheckSum.toString(CheckSumType.MD5))); + boolean saveKeys = props.getBoolean("save.keys", true); + boolean reducerPerBucket = props.getBoolean("reducer.per.bucket", false); + + new VoldemortStoreBuilderJob(this.getId() + "-build-store", + props, + new VoldemortStoreBuilderConf(storeDef.getReplicationFactor(), + chunkSize, + tempDir, + outputPath, + inputPath, + cluster, + Lists.newArrayList(storeDef), + storeDef.getName(), + keySelection, + valSelection, + null, + null, + checkSumType, + saveKeys, + reducerPerBucket, + numChunks)).run(); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortRollbackJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortRollbackJob.java new file mode 100644 index 0000000000..5e378fbb3b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortRollbackJob.java @@ -0,0 +1,116 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.client.protocol.admin.AdminClientConfig; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.store.readonly.mr.utils.VoldemortUtils; +import voldemort.store.readonly.swapper.AdminStoreSwapper; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +import com.google.common.collect.Maps; + +/** + * This job rolls back to [ current version - 1 ]. This may not always work + * since some folks may start using the version feature to put their own version + * number. But till that doesn't happen, this will back. + * + */ +public class VoldemortRollbackJob extends AbstractJob { + + private final Logger log; + + private final Props props; + + private List storeNames; + + private List clusterUrls; + + public VoldemortRollbackJob(String name, Props props) throws IOException { + super(name); + this.props = props; + this.log = Logger.getLogger(name); + this.storeNames = VoldemortUtils.getCommaSeparatedStringValues(props.getString("store.name"), + "store names"); + this.clusterUrls = VoldemortUtils.getCommaSeparatedStringValues(props.getString("push.cluster"), + "cluster urls"); + + } + + @Override + public void run() throws Exception { + + // Go over every cluster and rollback one store at a time + for(String clusterUrl: clusterUrls) { + + AdminClient adminClient = null; + ExecutorService service = null; + try { + service = Executors.newCachedThreadPool(); + adminClient = new AdminClient(clusterUrl, new AdminClientConfig()); + Cluster cluster = adminClient.getAdminClientCluster(); + AdminStoreSwapper swapper = new AdminStoreSwapper(cluster, + service, + adminClient, + 1000 * props.getInt("timeout.seconds", + 24 * 60 * 60), + true, + true); + + // Get the current version for all stores on all nodes + Map> previousVersions = Maps.newHashMap(); + for(Node node: cluster.getNodes()) { + Map currentVersion = adminClient.getROCurrentVersion(node.getId(), + storeNames); + + log.info("Retrieving current version information on node " + node.getId()); + Map previousVersion = Maps.newHashMap(); + for(Entry entry: currentVersion.entrySet()) { + previousVersion.put(entry.getKey(), entry.getValue() - 1); + if(entry.getValue() == 0) { + throw new VoldemortException("Store '" + entry.getKey() + "' on node " + + node.getId() + + " does not have version to rollback to"); + } + } + previousVersions.put(node.getId(), previousVersion); + } + + // Swap one store at a time + for(String storeName: storeNames) { + for(Node node: cluster.getNodes()) { + log.info("Rolling back data on node " + node.getId() + " and for store " + + storeName + " to version " + + previousVersions.get(node.getId()).get(storeName)); + swapper.invokeRollback(storeName, + previousVersions.get(node.getId()).get(storeName)); + log.info("Successfully rolled back data on node " + node.getId() + + " and for store " + storeName); + } + } + } finally { + if(service != null) { + service.shutdownNow(); + service.awaitTermination(10, TimeUnit.SECONDS); + service = null; + } + if(adminClient != null) { + adminClient.stop(); + adminClient = null; + } + } + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortStoreBuilderJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortStoreBuilderJob.java new file mode 100644 index 0000000000..934c59c8ea --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortStoreBuilderJob.java @@ -0,0 +1,450 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.FileNotFoundException; +import java.io.InputStreamReader; +import java.util.List; +import java.util.Random; + +import org.apache.avro.mapred.AvroInputFormat; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.store.readonly.mr.AvroStoreBuilderMapper; +import voldemort.store.readonly.mr.HadoopStoreBuilder; +import voldemort.store.readonly.mr.VoldemortStoreBuilderMapper; +import voldemort.store.readonly.mr.serialization.JsonSequenceFileInputFormat; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; +import azkaban.common.utils.Props; + +/** + * Build a voldemort store from input data. + * + * @author jkreps + * + */ +public class VoldemortStoreBuilderJob extends AbstractHadoopJob { + + private VoldemortStoreBuilderConf conf; + private boolean isAvro; + + public VoldemortStoreBuilderJob(String name, Props props) throws Exception { + super(name, props); + this.conf = new VoldemortStoreBuilderConf(createJobConf(VoldemortStoreBuilderMapper.class), + props); + this.isAvro = false; + } + + public VoldemortStoreBuilderJob(String name, Props props, VoldemortStoreBuilderConf conf) + throws FileNotFoundException { + super(name, props); + this.conf = conf; + this.isAvro = false; + } + + public VoldemortStoreBuilderJob(String name, Props props, boolean isAvro) throws Exception { + super(name, props); + this.conf = new VoldemortStoreBuilderConf(createJobConf(VoldemortStoreBuilderMapper.class), + props); + this.isAvro = isAvro; + } + + public VoldemortStoreBuilderJob(String name, + Props props, + VoldemortStoreBuilderConf conf, + boolean isAvro) throws FileNotFoundException { + super(name, props); + this.conf = conf; + this.isAvro = isAvro; + } + + public static final class VoldemortStoreBuilderConf { + + private int replicationFactor; + private int chunkSize; + private Path tempDir; + private Path outputDir; + private Path inputPath; + private Cluster cluster; + private List storeDefs; + private String storeName; + private String keySelection; + private String valSelection; + private String keyTrans; + private String valTrans; + private CheckSumType checkSumType; + private boolean saveKeys; + private boolean reducerPerBucket; + private int numChunks = -1; + + private String recSchema; + private String keySchema; + private String valSchema; + + private String keyField; + private String valueField; + + public VoldemortStoreBuilderConf(int replicationFactor, + int chunkSize, + Path tempDir, + Path outputDir, + Path inputPath, + Cluster cluster, + List storeDefs, + String storeName, + String keySelection, + String valSelection, + String keyTrans, + String valTrans, + CheckSumType checkSumType, + boolean saveKeys, + boolean reducerPerBucket, + int numChunks) { + this.replicationFactor = replicationFactor; + this.chunkSize = chunkSize; + this.tempDir = tempDir; + this.outputDir = outputDir; + this.inputPath = inputPath; + this.cluster = cluster; + this.storeDefs = storeDefs; + this.storeName = storeName; + this.keySelection = keySelection; + this.valSelection = valSelection; + this.keyTrans = keyTrans; + this.valTrans = valTrans; + this.checkSumType = checkSumType; + this.saveKeys = saveKeys; + this.reducerPerBucket = reducerPerBucket; + this.numChunks = numChunks; + } + + // requires job conf in order to get files from the filesystem + public VoldemortStoreBuilderConf(JobConf configuration, Props props) throws Exception { + this(props.getInt("replication.factor", 2), + props.getInt("chunk.size", 1024 * 1024 * 1024), + new Path(props.getString("temp.dir", + "/tmp/vold-build-and-push-" + new Random().nextLong())), + new Path(props.getString("output.dir")), + new Path(props.getString("input.path")), + new ClusterMapper().readCluster(new InputStreamReader(new Path(props.getString("cluster.xml")).getFileSystem(configuration) + .open(new Path(props.getString("cluster.xml"))))), + new StoreDefinitionsMapper().readStoreList(new InputStreamReader(new Path(props.getString("stores.xml")).getFileSystem(configuration) + .open(new Path(props.getString("stores.xml"))))), + props.getString("store.name"), + props.getString("key.selection", null), + props.getString("value.selection", null), + props.getString("key.transformation.class", null), + props.getString("value.transformation.class", null), + CheckSum.fromString(props.getString("checksum.type", + CheckSum.toString(CheckSumType.MD5))), + props.getBoolean("save.keys", true), + props.getBoolean("reducer.per.bucket", false), + props.getInt("num.chunks", -1)); + } + + // new constructor to include the key val and record schema + + public VoldemortStoreBuilderConf(int replicationFactor, + int chunkSize, + Path tempDir, + Path outputDir, + Path inputPath, + Cluster cluster, + List storeDefs, + String storeName, + String keySelection, + String valSelection, + String keyTrans, + String valTrans, + CheckSumType checkSumType, + boolean saveKeys, + boolean reducerPerBucket, + int numChunks, + String keyField, + String valueField, + String recSchema, + String keySchema, + String valSchema) { + this.replicationFactor = replicationFactor; + this.chunkSize = chunkSize; + this.tempDir = tempDir; + this.outputDir = outputDir; + this.inputPath = inputPath; + this.cluster = cluster; + this.storeDefs = storeDefs; + this.storeName = storeName; + this.keySelection = keySelection; + this.valSelection = valSelection; + this.keyTrans = keyTrans; + this.valTrans = valTrans; + this.checkSumType = checkSumType; + this.saveKeys = saveKeys; + this.reducerPerBucket = reducerPerBucket; + this.numChunks = numChunks; + + this.keyField = keyField; + this.valueField = valueField; + this.recSchema = recSchema; + this.keySchema = keySchema; + this.valSchema = valSchema; + + } + + // requires job conf in order to get files from the filesystem + public VoldemortStoreBuilderConf(JobConf configuration, + Props props, + String keyField, + String valueField, + String recSchema, + String keySchema, + String valSchema) throws Exception { + this(props.getInt("replication.factor", 2), + props.getInt("chunk.size", 1024 * 1024 * 1024), + new Path(props.getString("temp.dir", + "/tmp/vold-build-and-push-" + new Random().nextLong())), + new Path(props.getString("output.dir")), + new Path(props.getString("input.path")), + new ClusterMapper().readCluster(new InputStreamReader(new Path(props.getString("cluster.xml")).getFileSystem(configuration) + .open(new Path(props.getString("cluster.xml"))))), + new StoreDefinitionsMapper().readStoreList(new InputStreamReader(new Path(props.getString("stores.xml")).getFileSystem(configuration) + .open(new Path(props.getString("stores.xml"))))), + props.getString("store.name"), + props.getString("key.selection", null), + props.getString("value.selection", null), + props.getString("key.transformation.class", null), + props.getString("value.transformation.class", null), + CheckSum.fromString(props.getString("checksum.type", + CheckSum.toString(CheckSumType.MD5))), + props.getBoolean("save.keys", true), + props.getBoolean("reducer.per.bucket", false), + props.getInt("num.chunks", -1), + keyField, + valueField, + recSchema, + keySchema, + valSchema); + } + + public int getReplicationFactor() { + return replicationFactor; + } + + public int getChunkSize() { + return chunkSize; + } + + public Path getTempDir() { + return tempDir; + } + + public Path getOutputDir() { + return outputDir; + } + + public Path getInputPath() { + return inputPath; + } + + public String getStoreName() { + return storeName; + } + + public String getKeySelection() { + return keySelection; + } + + public String getValSelection() { + return valSelection; + } + + public String getKeyTrans() { + return keyTrans; + } + + public String getValTrans() { + return valTrans; + } + + public Cluster getCluster() { + return cluster; + } + + public List getStoreDefs() { + return storeDefs; + } + + public CheckSumType getCheckSumType() { + return checkSumType; + } + + public boolean getSaveKeys() { + return saveKeys; + } + + public boolean getReducerPerBucket() { + return reducerPerBucket; + } + + public int getNumChunks() { + return numChunks; + } + + public String getRecSchema() { + return recSchema; + } + + public void setRecSchema(String recSchema) { + this.recSchema = recSchema; + } + + public String getKeySchema() { + return keySchema; + } + + public void setKeySchema(String keySchema) { + this.keySchema = keySchema; + } + + public String getValSchema() { + return valSchema; + } + + public void setValSchema(String valSchema) { + this.valSchema = valSchema; + } + + public String getValueField() { + return valueField; + } + + public void setValueField(String valueField) { + this.valueField = valueField; + } + + public String getKeyField() { + return keyField; + } + + public void setKeyField(String keyField) { + this.keyField = keyField; + } + + } + + @Override + public void run() throws Exception { + JobConf configuration = this.createJobConf(VoldemortStoreBuilderMapper.class); + if(isAvro) { + String recSchema = conf.getRecSchema(); + String keySchema = conf.getKeySchema(); + String valSchema = conf.getValSchema(); + + String keyField = conf.getKeyField(); + String valueField = conf.getValueField(); + + configuration.set("avro.rec.schema", recSchema); + configuration.set("avro.key.schema", keySchema); + configuration.set("avro.val.schema", valSchema); + + configuration.set("avro.key.field", keyField); + configuration.set("avro.value.field", valueField); + } + int chunkSize = conf.getChunkSize(); + Path tempDir = conf.getTempDir(); + Path outputDir = conf.getOutputDir(); + Path inputPath = conf.getInputPath(); + Cluster cluster = conf.getCluster(); + List storeDefs = conf.getStoreDefs(); + String storeName = conf.getStoreName(); + CheckSumType checkSumType = conf.getCheckSumType(); + boolean saveKeys = conf.getSaveKeys(); + boolean reducerPerBucket = conf.getReducerPerBucket(); + + StoreDefinition storeDef = null; + for(StoreDefinition def: storeDefs) + if(storeName.equals(def.getName())) + storeDef = def; + if(storeDef == null) + throw new IllegalArgumentException("Store '" + storeName + "' not found."); + + FileSystem fs = outputDir.getFileSystem(configuration); + if(fs.exists(outputDir)) { + info("Deleting previous output in " + outputDir + " for building store " + storeName); + fs.delete(outputDir, true); + } + + HadoopStoreBuilder builder = null; + + if(isAvro) { + + if(conf.getNumChunks() == -1) { + builder = new HadoopStoreBuilder(configuration, + + AvroStoreBuilderMapper.class, + (Class) AvroInputFormat.class, + cluster, + storeDef, + chunkSize, + tempDir, + outputDir, + inputPath, + checkSumType, + saveKeys, + reducerPerBucket); + } else { + builder = new HadoopStoreBuilder(configuration, + AvroStoreBuilderMapper.class, + (Class) AvroInputFormat.class, + cluster, + storeDef, + tempDir, + outputDir, + inputPath, + checkSumType, + saveKeys, + reducerPerBucket, + conf.getNumChunks()); + } + + builder.buildAvro(); + return; + } + + if(conf.getNumChunks() == -1) { + builder = new HadoopStoreBuilder(configuration, + VoldemortStoreBuilderMapper.class, + JsonSequenceFileInputFormat.class, + cluster, + storeDef, + chunkSize, + tempDir, + outputDir, + inputPath, + checkSumType, + saveKeys, + reducerPerBucket); + } else { + builder = new HadoopStoreBuilder(configuration, + VoldemortStoreBuilderMapper.class, + JsonSequenceFileInputFormat.class, + cluster, + storeDef, + tempDir, + outputDir, + inputPath, + checkSumType, + saveKeys, + reducerPerBucket, + conf.getNumChunks()); + } + + builder.build(); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapJob.java new file mode 100644 index 0000000000..93f4a8bd60 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapJob.java @@ -0,0 +1,200 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.client.protocol.admin.AdminClient; +import voldemort.client.protocol.admin.AdminClientConfig; +import voldemort.cluster.Cluster; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.store.readonly.swapper.AdminStoreSwapper; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +/* + * Call voldemort to swap the current store for the specified store + */ +public class VoldemortSwapJob extends AbstractJob { + + private final Props _props; + private VoldemortSwapConf swapConf; + + public VoldemortSwapJob(String id, Props props) throws IOException { + super(id); + _props = props; + swapConf = new VoldemortSwapConf(_props); + } + + public VoldemortSwapJob(String id, Props props, VoldemortSwapConf conf) throws IOException { + super(id); + _props = props; + swapConf = conf; + } + + public static final class VoldemortSwapConf { + + private Cluster cluster; + private String dataDir; + private String storeName; + private int httpTimeoutMs; + private long pushVersion; + private int maxBackoffDelayMs = 60 * 1000; + private boolean rollback = false; + + public VoldemortSwapConf(Props props) throws IOException { + this(HadoopUtils.readCluster(props.getString("cluster.xml"), new Configuration()), + props.getString("data.dir"), + props.get("store.name"), + 1000 * props.getInt("http.timeout.seconds", 24 * 60 * 60), // for + // backwards + // compatibility + // http timeout = + // admin client + // timeout + props.getLong("push.version", -1L)); + } + + public VoldemortSwapConf(Cluster cluster, + String dataDir, + String storeName, + int httpTimeoutMs, + long pushVersion, + int maxBackoffDelayMs, + boolean rollback) { + this.cluster = cluster; + this.dataDir = dataDir; + this.storeName = storeName; + this.httpTimeoutMs = httpTimeoutMs; + this.pushVersion = pushVersion; + this.maxBackoffDelayMs = maxBackoffDelayMs; + this.rollback = rollback; + } + + public VoldemortSwapConf(Cluster cluster, + String dataDir, + String storeName, + int httpTimeoutMs, + long pushVersion) { + this.cluster = cluster; + this.dataDir = dataDir; + this.storeName = storeName; + this.httpTimeoutMs = httpTimeoutMs; + this.pushVersion = pushVersion; + } + + public Cluster getCluster() { + return cluster; + } + + public String getDataDir() { + return dataDir; + } + + public String getStoreName() { + return storeName; + } + + public int getHttpTimeoutMs() { + return httpTimeoutMs; + } + + public long getPushVersion() { + return pushVersion; + } + + public int getMaxBackoffDelayMs() { + return maxBackoffDelayMs; + } + + public boolean getRollback() { + return rollback; + } + } + + public void run() throws Exception { + String dataDir = swapConf.getDataDir(); + String storeName = swapConf.getStoreName(); + int httpTimeoutMs = swapConf.getHttpTimeoutMs(); + long pushVersion = swapConf.getPushVersion(); + Cluster cluster = swapConf.getCluster(); + ExecutorService executor = Executors.newCachedThreadPool(); + + // Read the hadoop configuration settings + JobConf conf = new JobConf(); + Path dataPath = new Path(dataDir); + dataDir = dataPath.makeQualified(FileSystem.get(conf)).toString(); + + /* + * Set the protocol according to config: webhdfs if its enabled + * Otherwise use hftp. + */ + Configuration hadoopConfig = new Configuration(); + String protocolName = hadoopConfig.get("dfs.webhdfs.enabled"); + String protocolPort = ""; + if(hadoopConfig.get("dfs.http.address").split(":").length >= 2) + protocolPort = hadoopConfig.get("dfs.http.address").split(":")[1]; + protocolName = (protocolName == null) ? "hftp" : "webhdfs"; + + /* + * Replace the default protocol and port with the one derived as above + */ + String existingProtocol = ""; + String existingPort = ""; + String[] pathComponents = dataDir.split(":"); + if(pathComponents.length >= 3) { + existingProtocol = pathComponents[0]; + existingPort = pathComponents[2].split("/")[0]; + } + info("Existing protocol = " + existingProtocol + " and port = " + existingPort); + if(protocolName.length() > 0 && protocolPort.length() > 0) { + dataDir = dataDir.replaceFirst(existingProtocol, protocolName); + dataDir = dataDir.replaceFirst(existingPort, protocolPort); + } + info("dfs.webhdfs.enabled = " + hadoopConfig.get("dfs.webhdfs.enabled") + + " and new protocol = " + protocolName + " and port = " + protocolPort); + + // Create admin client + AdminClient client = new AdminClient(cluster, + new AdminClientConfig().setMaxConnectionsPerNode(cluster.getNumberOfNodes()) + .setAdminConnectionTimeoutSec(httpTimeoutMs / 1000) + .setMaxBackoffDelayMs(swapConf.getMaxBackoffDelayMs())); + + if(pushVersion == -1L) { + + // Need to retrieve max version + ArrayList stores = new ArrayList(); + stores.add(storeName); + Map pushVersions = client.getROMaxVersion(stores); + + if(pushVersions == null || !pushVersions.containsKey(storeName)) { + throw new RuntimeException("Push version could not be determined for store " + + storeName); + } + pushVersion = pushVersions.get(storeName); + pushVersion++; + } + + // do the swap + info("Initiating swap of " + storeName + " with dataDir:" + dataDir); + AdminStoreSwapper swapper = new AdminStoreSwapper(cluster, + executor, + client, + httpTimeoutMs, + swapConf.getRollback(), + swapConf.getRollback()); + swapper.swapStoreData(storeName, dataDir, pushVersion); + info("Swap complete."); + executor.shutdownNow(); + executor.awaitTermination(10, TimeUnit.SECONDS); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapperUtils.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapperUtils.java new file mode 100644 index 0000000000..a0685aba53 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapperUtils.java @@ -0,0 +1,72 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; +import java.net.HttpURLConnection; +import java.net.URL; +import java.net.URLEncoder; + +import voldemort.cluster.Node; + +@Deprecated +public class VoldemortSwapperUtils { + + public static void doSwap(String storeName, Node node, String destinationDir) + throws IOException { + // construct data operation = swap + String data = URLEncoder.encode("operation", "UTF-8") + "=" + + URLEncoder.encode("swap", "UTF-8"); + // add index = indexFileName + data += "&" + URLEncoder.encode("index", "UTF-8") + "=" + + URLEncoder.encode(getIndexDestinationFile(node.getId(), destinationDir), "UTF-8"); + // add data = dataFileName + data += "&" + URLEncoder.encode("data", "UTF-8") + "=" + + URLEncoder.encode(getDataDestinationFile(node.getId(), destinationDir), "UTF-8"); + // add store= storeName + data += "&" + URLEncoder.encode("store", "UTF-8") + "=" + + URLEncoder.encode(storeName, "UTF-8"); + + // Send data + URL url = new URL("http://" + node.getHost() + ":" + node.getHttpPort() + "/read-only/mgmt"); + System.out.println("swapping node:" + node.getId() + " with url:" + url.toString() + + " data:" + data); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("POST"); + connection.setRequestProperty("Content-Type", "application/x-www-form-urlencoded"); + + connection.setRequestProperty("Content-Length", "" + + Integer.toString(data.getBytes().length)); + connection.setRequestProperty("Content-Language", "en-US"); + + connection.setUseCaches(false); + connection.setDoInput(true); + connection.setDoOutput(true); + + OutputStreamWriter wr = new OutputStreamWriter(connection.getOutputStream()); + wr.write(data); + wr.flush(); + wr.close(); + // Get Response + InputStream is = connection.getInputStream(); + BufferedReader rd = new BufferedReader(new InputStreamReader(is)); + String line; + StringBuffer response = new StringBuffer(); + while((line = rd.readLine()) != null) { + response.append(line); + response.append('\r'); + } + System.out.println("doSwap Completed for " + node + " Response:" + response.toString()); + rd.close(); + } + + public static String getIndexDestinationFile(int nodeId, String destinationDir) { + return destinationDir + "/" + "node-" + nodeId + ".index"; + } + + public static String getDataDestinationFile(int nodeId, String destinationDir) { + return destinationDir + "/" + "node-" + nodeId + ".data"; + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonConfigurable.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonConfigurable.java new file mode 100644 index 0000000000..e59df9d056 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonConfigurable.java @@ -0,0 +1,76 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; + +import voldemort.serialization.json.JsonTypeSerializer; +import azkaban.common.utils.Utils; + +/** + * Base class for a JsonMapper or JsonReducer with a few basic fields + * + * @author jkreps + * + */ +public abstract class JsonConfigurable implements JobConfigurable, Closeable { + + private volatile boolean _isConfigured = false; + private JsonTypeSerializer _inputKeySerializer; + private JsonTypeSerializer _inputValueSerializer; + private JsonTypeSerializer _outputKeySerializer; + private JsonTypeSerializer _outputValueSerializer; + + public void close() throws IOException { + + } + + public JsonTypeSerializer getInputKeySerializer() { + return _inputKeySerializer; + } + + public JsonTypeSerializer getInputValueSerializer() { + return _inputValueSerializer; + } + + public JsonTypeSerializer getOutputKeySerializer() { + return _outputKeySerializer; + } + + public JsonTypeSerializer getOutputValueSerializer() { + return _outputValueSerializer; + } + + protected void setInputKeySerializer(JsonTypeSerializer inputKeySerializer) { + _inputKeySerializer = Utils.nonNull(inputKeySerializer); + } + + protected void setInputValueSerializer(JsonTypeSerializer inputValueSerializer) { + _inputValueSerializer = Utils.nonNull(inputValueSerializer); + } + + protected void setOutputKeySerializer(JsonTypeSerializer outputKeySerializer) { + _outputKeySerializer = Utils.nonNull(outputKeySerializer); + } + + protected void setOutputValueSerializer(JsonTypeSerializer outputValueSerializer) { + _outputValueSerializer = Utils.nonNull(outputValueSerializer); + } + + protected void setConfigured(boolean isConfigured) { + _isConfigured = isConfigured; + } + + public boolean isConfigured() { + return _isConfigured; + } + + protected JsonTypeSerializer getSchemaFromJob(JobConf conf, String key) { + if(conf.get(key) == null) + throw new IllegalArgumentException("Missing required parameter '" + key + "' on job."); + return new JsonTypeSerializer(conf.get(key)); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonDeserializerComparator.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonDeserializerComparator.java new file mode 100644 index 0000000000..aa67a9dfea --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonDeserializerComparator.java @@ -0,0 +1,118 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.DataInputStream; +import java.io.IOException; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.InputBuffer; +import org.apache.hadoop.io.RawComparator; + +import voldemort.serialization.SerializationException; +import voldemort.serialization.json.JsonTypeSerializer; + +/** + * A hadoop RawComparator that deserializes first. Usefull for sorting JSON objects + * + * @author jay + * + */ +public class JsonDeserializerComparator implements RawComparator, + Configurable +{ + + /** + * Should be same as BytesWritable.Length + */ + private int LENGTH_BYTES = 4; + + private Configuration config; + private InputBuffer buffer = new InputBuffer(); + private DataInputStream dataInput = new DataInputStream(buffer); + private JsonTypeSerializer serializer; + + public Configuration getConf() + { + return this.config; + } + + public void setConf(Configuration config) + { + if (config.get("json.schema") == null) + throw new IllegalArgumentException("No schema has been set!"); + this.serializer = new JsonTypeSerializer(config.get("json.schema")); + } + + public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) + { + return compareBytes(b1, + s1 + LENGTH_BYTES, + l1 - LENGTH_BYTES, + b2, + s2 + LENGTH_BYTES, + l2 - LENGTH_BYTES); + } + + public int compareBytes(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) + { + if (serializer == null) + throw new SerializationException("No serializer has been set!"); + try + { + buffer.reset(b1, s1, l1); + Object key1 = serializer.toObject(dataInput); + + buffer.reset(b2, s2, l2); + Object key2 = serializer.toObject(dataInput); + + if (key1 instanceof Comparable) + { + return this.compareSerializedObjects(key1, key2); + } + else + { + return customCompare(key1, key2, serializer); + } + } + catch (IOException e) + { + throw new SerializationException(e); + } + } + + public int customCompare(Object key1, Object key2, JsonTypeSerializer serializer) + { + byte[] b1 = serializer.toBytes(key1); + byte[] b2 = serializer.toBytes(key2); + + return BytesWritable.Comparator.compareBytes(b1, 0, b1.length, b2, 0, b2.length); + } + + public int compare(BytesWritable o1, BytesWritable o2) + { + return this.compareBytes(o1.getBytes(), + 0, + o1.getLength(), + o2.getBytes(), + 0, + o2.getLength()); + } + + public int compareSerializedObjects(Object o1, Object o2) + { + if (o1 == o2) + return 0; + else if (o1 == null) + return -1; + else if (o2 == null) + return 1; + else if (o1.getClass() != o2.getClass()) + throw new IllegalArgumentException("Attempt to compare two items of different classes: " + + o1.getClass() + " and " + o2.getClass()); + else if (o1 instanceof Comparable) + return ((Comparable) o1).compareTo(o2); + + throw new IllegalArgumentException("Incomparable object type!"); + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonMapper.java new file mode 100644 index 0000000000..104d1d8bdf --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonMapper.java @@ -0,0 +1,58 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.IOException; + +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; + +/** + * A Mapper that handles JSON serialization + * + * @author jkreps + * + */ +public abstract class JsonMapper extends JsonConfigurable implements Mapper +{ + + public abstract void mapObjects(Object key, Object value, OutputCollector output, Reporter reporter) throws IOException; + + public void configure(JobConf conf) + { + setInputKeySerializer(getSchemaFromJob(conf, "mapper.input.key.schema")); + setInputValueSerializer(getSchemaFromJob(conf, "mapper.input.value.schema")); + setOutputKeySerializer(getSchemaFromJob(conf, "mapper.output.key.schema")); + setOutputValueSerializer(getSchemaFromJob(conf, "mapper.output.value.schema")); + + // set comparator for input Key Schema + if (conf.getBoolean("use.json.comparator", false)) + { + conf.setOutputKeyComparatorClass(JsonDeserializerComparator.class); + conf.set("json.schema", conf.get("mapper.output.key.schema")); + } + setConfigured(true); + } + + @SuppressWarnings("unchecked") + public void map(BytesWritable key, + BytesWritable value, + OutputCollector output, + Reporter reporter) throws IOException + { + if(!isConfigured()) + throw new IllegalStateException("JsonMapper's configure method wasn't called. Please make sure that super.configure() is called."); + + mapObjects(getInputKeySerializer().toObject(key.get()), + getInputValueSerializer().toObject(value.get()), + getOutputCollector(output), + reporter); + } + + protected OutputCollector getOutputCollector(OutputCollector output) + { + return new JsonOutputCollector(output, getOutputKeySerializer(), getOutputValueSerializer()); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonOutputCollector.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonOutputCollector.java new file mode 100644 index 0000000000..d2100ae06e --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonOutputCollector.java @@ -0,0 +1,44 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.IOException; + +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.OutputCollector; + +import voldemort.serialization.Serializer; +import azkaban.common.utils.Utils; + +/** + * Output collector that handles JSON serialization + * + * @author jkreps + * + */ +public class JsonOutputCollector implements OutputCollector { + + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final OutputCollector innerCollector; + + public JsonOutputCollector(OutputCollector innerCollector, + Serializer keySerializer, + Serializer valueSerializer) { + this.innerCollector = Utils.nonNull(innerCollector); + this.keySerializer = Utils.nonNull(keySerializer); + this.valueSerializer = Utils.nonNull(valueSerializer); + } + + public void collect(K key, V value) throws IOException { + innerCollector.collect(new BytesWritable(keySerializer.toBytes(key)), + new BytesWritable(valueSerializer.toBytes(value))); + } + + public Serializer getKeySerializer() { + return keySerializer; + } + + public Serializer getValueSerializer() { + return valueSerializer; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonReducer.java new file mode 100644 index 0000000000..466cec503b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonReducer.java @@ -0,0 +1,89 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.serialization.Serializer; + +/** + * Base reducer class that uses JSON serialization + * + * @author jkreps + * + */ +public abstract class JsonReducer extends JsonConfigurable implements + Reducer +{ + + public void configure(JobConf conf) + { + setInputKeySerializer(getSchemaFromJob(conf, "mapper.output.key.schema")); + setInputValueSerializer(getSchemaFromJob(conf, "mapper.output.value.schema")); + setOutputKeySerializer(getSchemaFromJob(conf, "reducer.output.key.schema")); + setOutputValueSerializer(getSchemaFromJob(conf, "reducer.output.value.schema")); + + // set comparator for input Key Schema + if (conf.getBoolean("use.json.comparator", false)) + { + conf.setOutputKeyComparatorClass(JsonDeserializerComparator.class); + conf.set("json.schema", conf.get("mapper.output.key.schema")); + } + setConfigured(true); + } + + public abstract void reduceObjects(Object key, + Iterator values, + OutputCollector collector, + Reporter reporter) throws IOException; + + public void reduce(BytesWritable key, + Iterator values, + OutputCollector output, + Reporter reporter) throws IOException + { + reduceObjects(getInputKeySerializer().toObject(key.get()), + new TranslatingIterator(getInputValueSerializer(), values), + getOutputCollector(output), + reporter); + } + + protected OutputCollector getOutputCollector(OutputCollector output) + { + return new JsonOutputCollector(output, getOutputKeySerializer(), getOutputValueSerializer()); + } + + private static class TranslatingIterator implements Iterator + { + + private final Serializer serializer; + private final Iterator inner; + + public TranslatingIterator(Serializer serializer, Iterator inner) + { + this.serializer = serializer; + this.inner = inner; + } + + public boolean hasNext() + { + return inner.hasNext(); + } + + public Object next() + { + return serializer.toObject(inner.next().get()); + } + + public void remove() + { + inner.remove(); + } + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileInputFormat.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileInputFormat.java new file mode 100644 index 0000000000..f0e26c1956 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileInputFormat.java @@ -0,0 +1,110 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.util.StringUtils; +import org.apache.log4j.Logger; + +import voldemort.store.readonly.mr.utils.HadoopUtils; + +/** + * Extends {@link SequenceFileInputFormat} to support our JSON based + * serialization format. + * + * Reads in a SequenceFile Read out the schema from Metadata and save it as keys + * in configuration. + *
        + *
      • "mapper.input.key.schema"
      • + *
      • "mapper.input.value.schema"
      • + *
      + * + * @author bbansal + * + */ +public class JsonSequenceFileInputFormat extends + SequenceFileInputFormat { + + protected static final Logger log = Logger.getLogger(JsonSequenceFileInputFormat.class.getName()); + + @Override + public RecordReader getRecordReader(InputSplit split, + JobConf conf, + Reporter reporter) + throws IOException { + String inputPathString = ((FileSplit) split).getPath().toUri().getPath(); + log.info("Input file path:" + inputPathString); + Path inputPath = new Path(inputPathString); + + SequenceFile.Reader reader = new SequenceFile.Reader(inputPath.getFileSystem(conf), + inputPath, + conf); + SequenceFile.Metadata meta = reader.getMetadata(); + + try { + Text keySchema = meta.get(new Text("key.schema")); + Text valueSchema = meta.get(new Text("value.schema")); + + if(0 == keySchema.getLength() || 0 == valueSchema.getLength()) { + throw new Exception(); + } + + // update Joboconf with schemas + conf.set("mapper.input.key.schema", keySchema.toString()); + conf.set("mapper.input.value.schema", valueSchema.toString()); + } catch(Exception e) { + throw new IOException("Failed to Load Schema from file:" + inputPathString + "\n"); + } + return super.getRecordReader(split, conf, reporter); + } + + @Override + protected FileStatus[] listStatus(JobConf job) throws IOException { + String dirs = job.get("mapred.input.dir", ""); + String[] list = StringUtils.split(dirs); + + List status = new ArrayList(); + for(int i = 0; i < list.length; i++) { + status.addAll(getAllSubFileStatus(job, new Path(list[i]))); + } + + return status.toArray(new FileStatus[0]); + } + + private List getAllSubFileStatus(JobConf inputConf, Path filterMemberPath) + throws IOException { + List list = new ArrayList(); + + FileSystem fs = filterMemberPath.getFileSystem(inputConf); + FileStatus[] subFiles = fs.listStatus(filterMemberPath); + + if(null != subFiles) { + if(fs.isDirectory(filterMemberPath)) { + for(FileStatus subFile: subFiles) { + if(!HadoopUtils.shouldPathBeIgnored(subFile.getPath())) { + list.addAll(getAllSubFileStatus(inputConf, subFile.getPath())); + } + } + } else { + if(subFiles.length > 0 && !HadoopUtils.shouldPathBeIgnored(subFiles[0].getPath())) { + list.add(subFiles[0]); + } + } + } + + return list; + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileOutputFormat.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileOutputFormat.java new file mode 100644 index 0000000000..2bd0ca1fd2 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileOutputFormat.java @@ -0,0 +1,101 @@ +package voldemort.store.readonly.mr.serialization; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.ReflectionUtils; + +import voldemort.serialization.json.JsonTypeDefinition; + +/** + * Copy of hadoop's SequenceFileOutputFormat modified to set the schema as metadata on + * output files + * + * @author jkreps + * + */ +public class JsonSequenceFileOutputFormat extends + SequenceFileOutputFormat +{ + + public RecordWriter getRecordWriter(FileSystem ignored, + JobConf job, + String name, + Progressable progress) + throws IOException + { + + // Shamelessly copy in hadoop code to allow us to set the metadata with our schema + + // get the path of the temporary output file + Path file = FileOutputFormat.getTaskOutputPath(job, name); + + FileSystem fs = file.getFileSystem(job); + CompressionType compressionType = CompressionType.BLOCK; + // find the right codec + Class codecClass = getOutputCompressorClass(job, DefaultCodec.class); + CompressionCodec codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, job); + + // set the schema metadata + /* begin jays code */ + SequenceFile.Metadata meta = new SequenceFile.Metadata(); + meta.set(new Text("key.schema"), + new Text(getSchema("reducer.output.key.schema", job))); + meta.set(new Text("value.schema"), new Text(getSchema("reducer.output.value.schema", + job))); + + final SequenceFile.Writer out = SequenceFile.createWriter(fs, + job, + file, + job.getOutputKeyClass(), + job.getOutputValueClass(), + compressionType, + codec, + progress, + meta); + /* end jays code */ + + return new RecordWriter() + { + + public void write(BytesWritable key, BytesWritable value) + throws IOException + { + + out.append(key, value); + } + + public void close(Reporter reporter) + throws IOException + { + out.close(); + } + }; + } + + private String getSchema(String prop, JobConf conf) + { + String schema = conf.get(prop); + if (schema == null) + throw new IllegalArgumentException("The required property '" + prop + + "' is not defined in the JobConf for this Hadoop job."); + // check that it is a valid schema definition + JsonTypeDefinition.fromJson(schema); + + return schema; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/AvroUtils.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/AvroUtils.java new file mode 100644 index 0000000000..6e61dce04c --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/AvroUtils.java @@ -0,0 +1,105 @@ +package voldemort.store.readonly.mr.utils; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +public class AvroUtils { + + /** + * Pull the schema off of the given file (if it is a file). If it is a + * directory, then pull schemas off of all subfiles, and check that they are + * all the same schema. If so, return that schema, otherwise throw an + * exception + * + * @param fs The filesystem to use + * @param path The path from which to get the schema + * @param checkSameSchema boolean flag to check all files in directory for + * same schema + * @return The schema of this file or all its subfiles + * @throws IOException + */ + + @SuppressWarnings({ "unchecked", "rawtypes" }) + private static Schema getSchemaFromPath(FileSystem fs, Path path, boolean checkSameSchema) { + + try { + if(fs.isFile(path)) { + BufferedInputStream inStream = null; + try { + inStream = new BufferedInputStream(fs.open(path)); + } catch(IOException e1) { + // TODO Auto-generated catch block + e1.printStackTrace(); + } + GenericDatumReader datum = new GenericDatumReader(); + + DataFileStream reader = null; + try { + reader = new DataFileStream(inStream, datum); + } catch(IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + return reader.getSchema(); + } else { + FileStatus[] statuses = null; + if(fs.isDirectory(path)) { + // this is a directory, get schemas from all subfiles + statuses = fs.listStatus(path); + } else { + // this is wildcard path, get schemas from all matched files + statuses = fs.globStatus(path); + } + if(statuses == null || statuses.length == 0) + throw new IllegalArgumentException("No files found in path pattern " + + path.toUri().getPath()); + List schemas = new ArrayList(); + for(FileStatus status: statuses) { + if(!HadoopUtils.shouldPathBeIgnored(status.getPath())) { + if(!checkSameSchema) { + // return first valid schema w/o checking all files + return getSchemaFromPath(fs, status.getPath(), checkSameSchema); + } + schemas.add(getSchemaFromPath(fs, status.getPath(), checkSameSchema)); + } + } + + // now check that all the schemas are the same + if(schemas.size() > 0) { + Schema schema = schemas.get(0); + for(int i = 1; i < schemas.size(); i++) + if(!schema.equals(schemas.get(i))) + throw new IllegalArgumentException("The directory " + + path.toString() + + " contains heterogenous schemas: found both '" + + schema.toString() + "' and '" + + schemas.get(i).toString() + "'."); + + return schema; + } else { + throw new IllegalArgumentException("No Valid metadata file found for Path:" + + path.toString()); + } + } + } catch(Exception e) { + // logger.error("failed to get metadata from path:" + path); + throw new RuntimeException(e); + } + + } + + public static Schema getAvroSchemaFromPath(Path path) throws IOException { + return getSchemaFromPath(path.getFileSystem(new Configuration()), path, true); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/EmailMessage.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/EmailMessage.java new file mode 100644 index 0000000000..458f291b0c --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/EmailMessage.java @@ -0,0 +1,186 @@ +package voldemort.store.readonly.mr.utils; + +import java.io.File; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Properties; + +import javax.activation.DataHandler; +import javax.activation.DataSource; +import javax.activation.FileDataSource; +import javax.mail.BodyPart; +import javax.mail.Message; +import javax.mail.MessagingException; +import javax.mail.Session; +import javax.mail.Transport; +import javax.mail.internet.InternetAddress; +import javax.mail.internet.MimeBodyPart; +import javax.mail.internet.MimeMessage; +import javax.mail.internet.MimeMultipart; + +public class EmailMessage { + private List _toAddress = new ArrayList(); + private String _mailHost; + private String _mailUser; + private String _mailPassword; + private String _subject; + private String _fromAddress; + private String _mimeType = "text/plain"; + private StringBuffer _body = new StringBuffer(); + + private ArrayList _attachments = new ArrayList(); + + public EmailMessage() { + this("localhost", "", ""); + } + + public EmailMessage(String host, String user, String password) { + _mailUser = user; + _mailHost = host; + _mailPassword = password; + } + + public EmailMessage setMailHost(String host) { + _mailHost = host; + return this; + } + + public EmailMessage setMailUser(String user) { + _mailUser = user; + return this; + } + + public EmailMessage setMailPassword(String password) { + _mailPassword = password; + return this; + } + + public EmailMessage addAllToAddress(Collection< ? extends String > addresses) { + _toAddress.addAll(addresses); + return this; + } + + public EmailMessage addToAddress(String address) { + _toAddress.add(address); + return this; + } + + public EmailMessage setSubject(String subject) { + _subject = subject; + return this; + } + + public EmailMessage setFromAddress(String fromAddress) { + _fromAddress = fromAddress; + return this; + } + + public EmailMessage addAttachment(File file) throws MessagingException { + return addAttachment(file.getName(), file); + } + + public EmailMessage addAttachment(String attachmentName, File file) throws MessagingException { + BodyPart attachmentPart = new MimeBodyPart(); + DataSource fileDataSource = new FileDataSource(file); + attachmentPart.setDataHandler(new DataHandler(fileDataSource)); + attachmentPart.setFileName(attachmentName); + _attachments.add(attachmentPart); + return this; + } + + public EmailMessage addAttachment(String attachmentName, InputStream stream) throws MessagingException { + BodyPart attachmentPart = new MimeBodyPart(stream); + attachmentPart.setFileName(attachmentName); + _attachments.add(attachmentPart); + return this; + } + + private void checkSettings() { + if (_mailHost == null) { + throw new RuntimeException("Mail host not set."); + } + + if (_mailUser == null) { + throw new RuntimeException("Mail user not set."); + } + + if (_mailPassword == null) { + throw new RuntimeException("Mail password not set."); + } + + if (_fromAddress == null || _fromAddress.length() == 0) { + throw new RuntimeException("From address not set."); + } + + if (_subject == null ) { + throw new RuntimeException("Subject cannot be null"); + } + + if (_toAddress.size() == 0) { + throw new RuntimeException("T"); + } + } + + public void sendEmail() throws MessagingException{ + checkSettings(); + Properties props = new Properties(); + props.setProperty("mail.transport.protocol", "smtp"); + props.put("mail.host", _mailHost); + props.put("mail.user", _mailUser); + props.put("mail.password", _mailPassword); + + Session session = Session.getDefaultInstance(props); + Message message = new MimeMessage(session); + InternetAddress from = new InternetAddress(_fromAddress, false); + message.setFrom(from); + for(String toAddr: _toAddress) + message.addRecipient(Message.RecipientType.TO, new InternetAddress(toAddr, false)); + message.setSubject(_subject); + message.setSentDate(new Date()); + + if ( _attachments.size() > 0 ) { + MimeMultipart multipart = new MimeMultipart("related"); + // Add attachments + for(BodyPart part: _attachments) { + multipart.addBodyPart(part); + } + + BodyPart messageBodyPart = new MimeBodyPart(); + messageBodyPart.setContent(_body.toString(), _mimeType); + multipart.addBodyPart(messageBodyPart); + + message.setContent(multipart); + } + else { + message.setContent(_body.toString(), _mimeType); + } + + Transport transport = session.getTransport(); + transport.connect(); + transport.sendMessage(message, message.getRecipients(Message.RecipientType.TO)); + transport.close(); + } + + public void setBody(String body) { + setBody(body, _mimeType); + } + + public void setBody(String body, String mimeType) { + _body = new StringBuffer(body); + _mimeType = mimeType; + } + + public EmailMessage setMimeType(String mimeType) { + _mimeType = mimeType; + return this; + } + + public EmailMessage println(Object str) { + _body.append(str); + + return this; + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/HadoopUtils.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/HadoopUtils.java new file mode 100644 index 0000000000..adccf3021a --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/HadoopUtils.java @@ -0,0 +1,985 @@ +package voldemort.store.readonly.mr.utils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.io.StringReader; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.log4j.Logger; +import org.joda.time.Period; + +import voldemort.cluster.Cluster; +import voldemort.serialization.json.JsonTypeDefinition; +import voldemort.serialization.json.JsonTypes; +import voldemort.store.StoreDefinition; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; +import azkaban.common.utils.Props; +import azkaban.common.utils.UndefinedPropertyException; + +/** + * Helper functions for Hadoop + * + * @author jkreps + * + */ +public class HadoopUtils { + + // Any date written with the pattern should be accepted by the regex. + public static String COMMON_FILE_DATE_PATTERN = "yyyy-MM-dd-HH-mm"; + public static String COMMON_FILE_DATE_REGEX = "\\d{4}-\\d{2}-\\d{2}-\\d{2}-\\d{2}"; + + private static Logger logger = Logger.getLogger(HadoopUtils.class); + private static Object cachedSerializable = null; + + public static FileSystem getFileSystem(Props props) { + if(!props.containsKey("hadoop.job.ugi")) + throw new RuntimeException("No parameter hadoop.job.ugi set!"); + return getFileSystem(props.getString("hadoop.job.ugi")); + } + + public static FileSystem getFileSystem(String user) { + Configuration conf = new Configuration(); + conf.set("hadoop.job.ugi", user); + try { + return FileSystem.get(conf); + } catch(IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Add the given object to the distributed cache for this job + * + * @param obj A Serializable object to add to the JobConf + * @param job The JobConf + */ + public static void setSerializableInCache(JobConf job, T serializable) { + try { + // TODO: MED /tmp should be changed by conf.getTempDir() or + // something + Path workDir = new Path(String.format("/tmp/%s/%s/_join.temporary", + job.getJobName(), + System.currentTimeMillis())); + + Path tempPath = new Path(workDir, "serializable.dat"); + tempPath.getFileSystem(job).deleteOnExit(tempPath); + job.set("serializables.file", tempPath.toUri().getPath()); + + ObjectOutputStream objectStream = new ObjectOutputStream(tempPath.getFileSystem(job) + .create(tempPath)); + objectStream.writeObject(serializable); + objectStream.close(); + + DistributedCache.addCacheFile(new URI(tempPath.toUri().getPath() + "#" + + tempPath.getName()), + job); + } catch(URISyntaxException e) { + throw new RuntimeException(e); + } catch(IOException e) { + throw new RuntimeException(e); + } + } + + public static Path getFilePathFromDistributedCache(String filename, Configuration conf) + throws IOException { + + Path[] paths = DistributedCache.getLocalCacheFiles(conf); + Path filePath = null; + File file = new File(filename); + + if(paths == null) { + // maybe we are in local mode and hadoop is a complete piece of + // shit that doesn't + // work in local mode + // check if maybe the file is just sitting there on the + // filesystem + if(file.exists()) + filePath = new Path(file.getAbsolutePath()); + } else { + for(Path path: paths) + if(path.getName().equals(file.getName())) + filePath = path; + } + + return filePath; + + } + + /** + * Get the FileInputStream from distributed cache + * + * @param conf the JobConf + * @return FileInputStream file input stream + * @throws IOException + */ + public static FileInputStream getFileInputStream(String filename, Configuration conf) { + try { + Path filePath = getFilePathFromDistributedCache(filename, conf); + + if(filePath == null) { + Path[] paths = DistributedCache.getLocalCacheFiles(conf); + throw new IllegalStateException("No cache file found by the name of '" + filename + + "', found only " + paths); + } + return new FileInputStream(filePath.toString()); + } catch(IOException e) { + throw new RuntimeException(e); + } + + } + + /** + * Get the given Serializable from the distributed cache as an Object + * + * @param conf The JobConf + * @return The Object that is read from cache + */ + public static Object readSerializableFromCache(Configuration conf) { + /* + * Cache the results of this operation, as this function may be called + * more than once by the same process (i.e., by combiners). + */ + if(HadoopUtils.cachedSerializable != null) + return HadoopUtils.cachedSerializable; + + try { + String filename = conf.get("serializables.file"); + if(filename == null) + return null; + + Path serializable = getFilePathFromDistributedCache(filename, conf); + + if(serializable == null) { + Path[] paths = DistributedCache.getLocalCacheFiles(conf); + throw new IllegalStateException("No serializable cache file found by the name of '" + + filename + "', found only " + paths); + } + ObjectInputStream stream = new ObjectInputStream(new FileInputStream(serializable.toString())); + Object obj = stream.readObject(); + stream.close(); + HadoopUtils.cachedSerializable = obj; + return obj; + } catch(IOException e) { + throw new RuntimeException(e); + } catch(ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public static Map getMetadataFromSequenceFile(String fileName) { + Path path = new Path(fileName); + try { + return getMetadataFromSequenceFile(path.getFileSystem(new Configuration()), path); + } catch(IOException e) { + throw new RuntimeException(e); + } + } + + public static Map getMetadataFromSequenceFile(FileSystem fs, String fileName) { + return getMetadataFromSequenceFile(fs, new Path(fileName)); + } + + /** + * Read the metadata from a hadoop SequenceFile + * + * @param fs The filesystem to read from + * @param fileName The file to read from + * @return The metadata from this file + */ + public static Map getMetadataFromSequenceFile(FileSystem fs, Path path) { + try { + Configuration conf = new Configuration(); + conf.setInt("io.file.buffer.size", 4096); + SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, new Configuration()); + SequenceFile.Metadata meta = reader.getMetadata(); + reader.close(); + TreeMap map = meta.getMetadata(); + Map values = new HashMap(); + for(Map.Entry entry: map.entrySet()) + values.put(entry.getKey().toString(), entry.getValue().toString()); + + return values; + } catch(IOException e) { + throw new RuntimeException(e); + } + } + + public static JsonSchema getSchemaFromPath(Path path) throws IOException { + return getSchemaFromPath(path.getFileSystem(new Configuration()), path, true); + } + + public static JsonSchema getSchemaFromPath(FileSystem fs, Path path) throws IOException { + return getSchemaFromPath(fs, path, true); + } + + /** + * Pull the schema off of the given file (if it is a file). If it is a + * directory, then pull schemas off of all subfiles, and check that they are + * all the same schema. If so, return that schema, otherwise throw an + * exception + * + * @param fs The filesystem to use + * @param path The path from which to get the schema + * @param checkSameSchema boolean flag to check all files in directory for + * same schema + * @return The schema of this file or all its subfiles + * @throws IOException + */ + public static JsonSchema getSchemaFromPath(FileSystem fs, Path path, boolean checkSameSchema) + throws IOException { + try { + if(fs.isFile(path)) { + // this is a normal file, get a schema from it + Map m = HadoopUtils.getMetadataFromSequenceFile(fs, path); + if(!m.containsKey("value.schema") || !m.containsKey("key.schema")) + throw new IllegalArgumentException("No schema found on file " + path.toString()); + return new JsonSchema(JsonTypeDefinition.fromJson(m.get("key.schema")), + JsonTypeDefinition.fromJson(m.get("value.schema"))); + } else { + FileStatus[] statuses = null; + if(fs.isDirectory(path)) { + // this is a directory, get schemas from all subfiles + statuses = fs.listStatus(path); + } else { + // this is wildcard path, get schemas from all matched files + statuses = fs.globStatus(path); + } + if(statuses == null || statuses.length == 0) + throw new IllegalArgumentException("No files found in path pattern " + + path.toUri().getPath()); + List schemas = new ArrayList(); + for(FileStatus status: statuses) { + if(!HadoopUtils.shouldPathBeIgnored(status.getPath())) { + if(!checkSameSchema) { + // return first valid schema w/o checking all files + return getSchemaFromPath(fs, status.getPath(), checkSameSchema); + } + schemas.add(getSchemaFromPath(fs, status.getPath(), checkSameSchema)); + } + } + + // now check that all the schemas are the same + if(schemas.size() > 0) { + JsonSchema schema = schemas.get(0); + for(int i = 1; i < schemas.size(); i++) + if(!schema.equals(schemas.get(i))) + throw new IllegalArgumentException("The directory " + + path.toString() + + " contains heterogenous schemas: found both '" + + schema.toString() + "' and '" + + schemas.get(i).toString() + "'."); + + return schema; + } else { + throw new IllegalArgumentException("No Valid metedata file found for Path:" + + path.toString()); + } + } + } catch(Exception e) { + logger.error("failed to get metadata from path:" + path); + throw new RuntimeException(e); + } + } + + public static String getRequiredString(Configuration conf, String name) { + String val = conf.get(name); + if(val == null) + throw new IllegalArgumentException("Missing required parameter '" + name + "'."); + else + return val; + } + + public static int getRequiredInt(Configuration conf, String name) { + return Integer.parseInt(getRequiredString(conf, name)); + } + + public static void copyInProps(Props props, Configuration conf, String... keys) { + for(String key: keys) + if(props.get(key) != null) + conf.set(key, props.get(key)); + } + + public static void copyInRequiredProps(Props props, Configuration conf, String... keys) { + for(String key: keys) + conf.set(key, props.getString(key)); + } + + /** + * Add all the properties in the Props to the given Configuration + * + * @param conf The Configuration + * @param props The Props + * @return The Configuration with all the new properties + */ + public static void copyInAllProps(Props props, Configuration conf) { + for(String key: props.keySet()) + conf.set(key, props.get(key)); + } + + public static void copyInLocalProps(Props props, Configuration conf) { + for(String key: props.localKeySet()) + conf.set(key, props.get(key)); + } + + public static Props loadHadoopProps(Props parent, File hadoopConfDir) { + // load hadoop properties + Configuration config = new Configuration(); + + config.addResource(new Path(new File(hadoopConfDir, "hadoop-default.xml").getAbsolutePath())); + config.addResource(new Path(new File(hadoopConfDir, "hadoop-site.xml").getAbsolutePath())); + + // copy to props + Props props = new Props(parent); + for(Entry entry: config) + props.put(entry.getKey(), config.get(entry.getKey())); + + return props; + } + + public static void setPropsInJob(Configuration conf, Props props) { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + try { + props.storeFlattened(output); + conf.set("azkaban.props", new String(output.toByteArray(), "UTF-8")); + } catch(IOException e) { + throw new RuntimeException("This is not possible!", e); + } + } + + public static Props getPropsFromJob(Configuration conf) { + String propsString = conf.get("azkaban.props"); + if(propsString == null) + throw new UndefinedPropertyException("The required property azkaban.props was not found in the Configuration."); + try { + ByteArrayInputStream input = new ByteArrayInputStream(propsString.getBytes("UTF-8")); + Properties properties = new Properties(); + properties.load(input); + return new Props(null, properties); + } catch(IOException e) { + throw new RuntimeException("This is not possible!", e); + } + } + + public static Cluster readCluster(String clusterFile, Configuration conf) throws IOException { + return new ClusterMapper().readCluster(new StringReader(readAsString(new Path(clusterFile)))); + } + + public static StoreDefinition readStoreDef(String storeFile, + String storeName, + Configuration conf) throws IOException { + + List stores = new StoreDefinitionsMapper().readStoreList(new StringReader(readAsString(new Path(storeFile)))); + for(StoreDefinition def: stores) { + if(def.getName().equals(storeName)) + return def; + } + throw new RuntimeException("Can't find store definition for store '" + storeName + "'."); + } + + public static String getFileFromCache(Configuration conf, String fileName) throws IOException { + if("local".equals(conf.get("mapred.job.tracker"))) { + // For local mode Distributed cache is not set. + // try getting the raw file path. + URI[] uris = DistributedCache.getCacheFiles(conf); + return getFileFromURIList(uris, fileName); + } else { + // For Distributed filesystem. + Path[] pathList = DistributedCache.getLocalCacheFiles(conf); + return getFileFromPathList(pathList, fileName); + } + } + + public static String getFileFromURIList(URI[] uris, String fileName) throws IOException { + for(URI uri: uris) { + if(uri.getPath().endsWith(fileName)) { + // uri matched + return uri.getPath(); + } + } + return null; + } + + public static String getFileFromPathList(Path[] pathList, String fileName) { + for(Path file: pathList) { + logger.info("getUriWithFragment path:" + file.toUri().getPath() + " fileName:" + + fileName); + if(file.getName().equals(fileName)) { + logger.info("FOUND getUriWithFragment path:" + file.toUri().getPath()); + return file.toUri().getPath(); + } + } + + return null; + } + + /** + * Find a jar that contains a class of the same name, if any. It will return + * a jar file, even if that is not the first thing on the class path that + * has a class with the same name. + * + * @param my_class the class to find. + * @return a jar file that contains the class, or null. + * @throws IOException + */ + public static String findContainingJar(Class my_class, ClassLoader loader) { + String class_file = my_class.getName().replaceAll("\\.", "/") + ".class"; + return findContainingJar(class_file, loader); + } + + public static List getFileNames(FileStatus[] statuses) { + List fileNames = new ArrayList(); + if(statuses == null) + return fileNames; + for(FileStatus status: statuses) + fileNames.add(status.getPath().getName()); + return fileNames; + } + + public static String findContainingJar(String fileName, ClassLoader loader) { + try { + for(Enumeration itr = loader.getResources(fileName); itr.hasMoreElements();) { + URL url = (URL) itr.nextElement(); + logger.info("findContainingJar finds url:" + url); + if("jar".equals(url.getProtocol())) { + String toReturn = url.getPath(); + if(toReturn.startsWith("file:")) { + toReturn = toReturn.substring("file:".length()); + } + toReturn = URLDecoder.decode(toReturn, "UTF-8"); + return toReturn.replaceAll("!.*$", ""); + } + } + } catch(IOException e) { + throw new RuntimeException(e); + } + return null; + } + + public static String printAllClassLoaderPaths(String fileName, ClassLoader loader) { + try { + for(Enumeration itr = loader.getResources(fileName); itr.hasMoreElements();) { + URL url = (URL) itr.nextElement(); + logger.info("findContainingJar finds url:" + url); + } + } catch(IOException e) { + throw new RuntimeException(e); + } + return null; + } + + public static Period parsePeriod(String periodStr) { + Matcher monthsFormat = Pattern.compile("[0-9][0-9]*M").matcher(periodStr); + Matcher daysFormat = Pattern.compile("[0-9][0-9]*d").matcher(periodStr); + Matcher hoursFormat = Pattern.compile("[0-9][0-9]*h").matcher(periodStr); + Matcher minutesFormat = Pattern.compile("[0-9][0-9]*m").matcher(periodStr); + + Period period = new Period(); + while(monthsFormat.find()) { + period = period.plusMonths(Integer.parseInt(monthsFormat.group() + .substring(0, + monthsFormat.group() + .length() - 1))); + } + while(daysFormat.find()) { + period = period.plusDays(Integer.parseInt(daysFormat.group() + .substring(0, + daysFormat.group() + .length() - 1))); + } + while(hoursFormat.find()) { + period = period.plusHours(Integer.parseInt(hoursFormat.group() + .substring(0, + hoursFormat.group() + .length() - 1))); + } + while(minutesFormat.find()) { + period = period.plusMinutes(Integer.parseInt(minutesFormat.group() + .substring(0, + minutesFormat.group() + .length() - 1))); + } + + return period; + } + + public static FileSystem getFileSystem(String hdfsUrl, boolean isLocal) throws IOException { + // Initialize fs + FileSystem fs; + if(isLocal) { + fs = FileSystem.getLocal(new Configuration()); + } else { + fs = new DistributedFileSystem(); + try { + fs.initialize(new URI(hdfsUrl), new Configuration()); + } catch(URISyntaxException e) { + throw new IllegalArgumentException(e); + } + } + return fs; + } + + /** + * Given a directory path, return the paths of all directories in that tree + * that have no sub-directories. + * + * @param directory + * @return + */ + public static List getLowestLevelDirectories(FileSystem fs, + Path directory, + PathFilter pathFilter) throws IOException { + List lowestLevelDirectories = new ArrayList(); + + if(hasSubDirectories(fs, directory)) { + // recurse on each of this directory's sub-directories, ignoring any + // files in the + // directory + FileStatus[] statuses = fs.listStatus(directory); + for(FileStatus status: statuses) { + if(status.isDir()) { + lowestLevelDirectories.addAll(getLowestLevelDirectories(fs, + status.getPath(), + pathFilter)); + } + } + } else if(pathFilter == null || pathFilter.accept(directory)) { + // this directory has no sub-directories, and either there is no + // filter or it passes the + // filter, so add it and return + lowestLevelDirectories.add(directory.toString()); + } + + return lowestLevelDirectories; + } + + /** + * Given a string representation of a directory path, check whether or not + * the directory has any sub-directories + * + * @param fs + * @param directory + * @return true iff the directory has at least one sub-directory + * @throws IOException + */ + private static boolean hasSubDirectories(FileSystem fs, Path directory) throws IOException { + FileStatus[] statuses = fs.listStatus(directory); + + if(statuses == null) + return false; + + for(FileStatus status: statuses) { + if(status != null && status.isDir() && !shouldPathBeIgnored(status.getPath())) { + // we have found a subDirectory + return true; + } + } + // we are done looping through the directory and found no subDirectories + return false; + } + + public static JobConf addAllSubPaths(JobConf conf, Path path) throws IOException { + if(shouldPathBeIgnored(path)) { + throw new IllegalArgumentException(String.format("Path[%s] should be ignored.", path)); + } + + final FileSystem fs = path.getFileSystem(conf); + + if(fs.exists(path)) { + for(FileStatus status: fs.listStatus(path)) { + if(!shouldPathBeIgnored(status.getPath())) { + if(status.isDir()) { + addAllSubPaths(conf, status.getPath()); + } else { + FileInputFormat.addInputPath(conf, status.getPath()); + } + } + } + } + + return conf; + } + + /** + * Check if the path should be ignored. Currently only paths with "_log" are + * ignored. + * + * @param path + * @return + * @throws IOException + */ + public static boolean shouldPathBeIgnored(Path path) throws IOException { + return path.getName().startsWith("_"); + } + + public static Map getMapByPrefix(Configuration conf, String prefix) { + Map values = new HashMap(); + for(Entry entry: conf) { + if(entry.getKey().startsWith(prefix)) + values.put(entry.getKey().substring(prefix.length()), entry.getValue()); + } + return values; + } + + public static void saveProps(Props props, String file) throws IOException { + Path path = new Path(file); + + FileSystem fs = null; + if(props.containsKey("hadoop.job.ugi")) { + fs = getFileSystem(props); + } else { + fs = path.getFileSystem(new Configuration()); + } + + saveProps(fs, props, file); + } + + public static void saveProps(FileSystem fs, Props props, String file) throws IOException { + Path path = new Path(file); + + // create directory if it does not exist. + Path parent = path.getParent(); + if(!fs.exists(parent)) + fs.mkdirs(parent); + + // write out properties + OutputStream output = fs.create(path); + try { + props.storeFlattened(output); + } finally { + output.close(); + } + } + + public static Props readProps(String file) throws IOException { + Path path = new Path(file); + FileSystem fs = path.getFileSystem(new Configuration()); + if(fs.exists(path)) { + InputStream input = fs.open(path); + try { + // wrap it up in another layer so that the user can override + // properties + Props p = new Props(null, input); + return new Props(p); + } finally { + input.close(); + } + } else { + return new Props(); + } + } + + public static String readAsString(Path path) { + InputStream input = null; + try { + FileSystem fs = path.getFileSystem(new Configuration()); + input = fs.open(path); + return IOUtils.toString(input); + } catch(IOException e) { + throw new RuntimeException(e); + } finally { + IOUtils.closeQuietly(input); + } + } + + public static boolean mkdirs(String pathName) throws IOException { + Path path = new Path(pathName); + FileSystem fs = path.getFileSystem(new Configuration()); + return fs.mkdirs(path); + } + + public static void deletePathIfExists(JobConf conf, String stepOutputPath) throws IOException { + Path path = new Path(stepOutputPath); + FileSystem fs = path.getFileSystem(conf); + if(fs.exists(path)) { + fs.delete(path, true); + } + } + + /** + * Tag the BytesWritable with an integer at the END + */ + public static void appendTag(BytesWritable writable, int tag) { + int size = writable.getLength(); + + if(writable.getCapacity() < size + 4) { + // BytesWritable preserves old values + writable.setCapacity(size + 4); + } + + ByteUtils.writeInt(writable.getBytes(), tag, size); + writable.setSize(size + 4); + } + + /** + * read and return integer from the END of BytesWritable The tag bytes are + * NOT removed + */ + public static int readTag(BytesWritable readable) { + return ByteUtils.readInt(readable.getBytes(), readable.getLength() - 4); + } + + /** + * creates default data for given schema is needed for mappers/reducers + * which tries to handle different schema. + * + * Outputs
      + *
      + * Map : outputs default value for each subType
      + * List : output empty list
      + * JsonTypes: default 0 or '' empty strings + */ + public static Object createDefaultData(Object typeSchema) { + if(typeSchema instanceof List) { + ArrayList list = new ArrayList(0); + return list; + } else if(typeSchema instanceof Map) { + HashMap map = new HashMap(); + for(Map.Entry typeEntry: ((Map) typeSchema).entrySet()) { + map.put(typeEntry.getKey(), createDefaultData(typeEntry.getValue())); + } + return map; + } else if(typeSchema instanceof JsonTypes) { + return createDefaultJsonData((JsonTypes) typeSchema); + } + + throw new RuntimeException("Invlaid schema type:" + typeSchema); + } + + private static Object createDefaultJsonData(JsonTypes type) { + + if(JsonTypes.BOOLEAN.equals(type)) + return false; + else if(JsonTypes.DATE.equals(type)) + return new Date(); + else if(JsonTypes.FLOAT32.equals(type) || JsonTypes.FLOAT64.equals(type) + || JsonTypes.INT8.equals(type) || JsonTypes.INT16.equals(type) + || JsonTypes.INT32.equals(type)) + return 0; + else if(JsonTypes.BYTES.equals(type)) { + byte[] data = new byte[0]; + return data; + } else if(JsonTypes.STRING.equals(type)) { + return ""; + } + + throw new RuntimeException("Invalid JsonType:" + type); + } + + /** + * Looks for the latest (the alphabetically greatest) path contained in the + * given directory that passes the specified regex pattern. + * + * @param fs The file system + * @param directory The directory that will contain the versions + * @param acceptRegex The String pattern + * @return + * @throws IOException + */ + public static Path getLatestVersionedPath(FileSystem fs, Path directory, String acceptRegex) + throws IOException { + final String pattern = acceptRegex != null ? acceptRegex : "\\S+"; + + PathFilter filter = new PathFilter() { + + @Override + public boolean accept(Path arg0) { + return !arg0.getName().startsWith("_") && Pattern.matches(pattern, arg0.getName()); + } + }; + + FileStatus[] statuses = fs.listStatus(directory, filter); + + if(statuses == null || statuses.length == 0) { + return null; + } + + Arrays.sort(statuses); + + return statuses[statuses.length - 1].getPath(); + } + + /** + * Looks for the latest (the alphabetically greatest) path contained in the + * given directory that passes the specified regex pattern "\\S+" for all + * non spaced words. + * + * @param fs The file system + * @param directory The directory that will contain the versions + * @return + * @throws IOException + */ + public static Path getLatestVersionedPath(FileSystem fs, Path directory) throws IOException { + return getLatestVersionedPath(fs, directory, null); + } + + /** + * Does the same thing as getLatestVersionedPath, but checks to see if the + * directory contains #LATEST. If it doesn't, it just returns what was + * passed in. + * + * @param fs + * @param directory + * @return + * @throws IOException + */ + public static Path getSanitizedPath(FileSystem fs, Path directory, String acceptRegex) + throws IOException { + if(directory.getName().endsWith("#LATEST")) { + // getparent strips out #LATEST + return getLatestVersionedPath(fs, directory.getParent(), acceptRegex); + } + + return directory; + } + + public static Path getSanitizedPath(Path path) throws IOException { + return getSanitizedPath(path.getFileSystem(new Configuration()), path); + } + + /** + * Does the same thing as getLatestVersionedPath, but checks to see if the + * directory contains #LATEST. If it doesn't, it just returns what was + * passed in. + * + * @param fs + * @param directory + * @return + * @throws IOException + */ + public static Path getSanitizedPath(FileSystem fs, Path directory) throws IOException { + if(directory.getName().endsWith("#LATEST")) { + // getparent strips out #LATEST + return getLatestVersionedPath(fs, directory.getParent(), null); + } + + return directory; + } + + /** + * Easily cleans up old data (alphabetically least) paths that is accepted + * by the regex. + * + * @param fs The file system + * @param directory The directory that will contain the versions + * @param acceptRegex The String pattern + * @param backupNumber The number of versions we should keep. Otherwise + * we'll clean up. + * @return + * @throws IOException + */ + public static void cleanupOlderVersions(FileSystem fs, + Path directory, + final String acceptRegex, + int backupNumber) throws IOException { + if(backupNumber < 1) { + logger.error("Number of versions must be 1 or greater"); + return; + } + + PathFilter filter = new PathFilter() { + + @Override + public boolean accept(Path arg0) { + return !arg0.getName().startsWith("_") + && Pattern.matches(acceptRegex, arg0.getName()); + } + }; + + FileStatus[] statuses = fs.listStatus(directory, filter); + if(statuses == null) { + logger.info("No backup files found"); + return; + } + + Arrays.sort(statuses); + + int lastIndex = statuses.length - backupNumber; + for(int i = 0; i < lastIndex; ++i) { + logger.info("Deleting " + statuses[i].getPath()); + fs.delete(statuses[i].getPath(), true); + } + } + + public static void cleanupOlderVersions(FileSystem fs, Path directory, int backupNumber) + throws IOException { + cleanupOlderVersions(fs, directory, "\\S+", backupNumber); + } + + /** + * Move the file from one place to another. Unlike the raw Hadoop API this + * will throw an exception if it fails. Like the Hadoop api it will fail if + * a file exists in the destination. + * + * @param fs The filesystem + * @param from The source file to move + * @param to The destination location + * @throws IOException + */ + public static void move(FileSystem fs, Path from, Path to) throws IOException { + boolean success = fs.rename(from, to); + if(!success) + throw new RuntimeException("Failed to move " + from + " to " + to); + } + + /** + * Move the give file to the given location. Delete any existing file in + * that location. Use the temp directory to make the operation as + * transactional as possible. Throws an exception if the move fails. + * + * @param fs The filesystem + * @param from The source file + * @param to The destination file + * @param temp A temp directory to use + * @throws IOException + */ + public static void replaceFile(FileSystem fs, Path from, Path to, Path temp) throws IOException { + fs.delete(temp, true); + move(fs, to, temp); + try { + move(fs, from, to); + fs.delete(temp, true); + } catch(IOException e) { + // hmm something went wrong, attempt to restore + fs.rename(temp, to); + throw e; + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/JsonSchema.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/JsonSchema.java new file mode 100644 index 0000000000..aced7fcf52 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/JsonSchema.java @@ -0,0 +1,49 @@ +package voldemort.store.readonly.mr.utils; + +import java.util.Arrays; + +import voldemort.serialization.json.JsonTypeDefinition; +import azkaban.common.utils.Utils; + +public class JsonSchema { + + private final JsonTypeDefinition key; + private final JsonTypeDefinition value; + + public JsonSchema(JsonTypeDefinition key, JsonTypeDefinition value) { + super(); + this.key = Utils.nonNull(key); + this.value = Utils.nonNull(value); + } + + public JsonTypeDefinition getKeyType() { + return key; + } + + public JsonTypeDefinition getValueType() { + return value; + } + + @Override + public int hashCode() { + return Arrays.hashCode(new Object[] { key, value }); + } + + @Override + public boolean equals(Object o) { + if(o == null) + return false; + if(o == this) + return true; + if(o.getClass() != JsonSchema.class) + return false; + JsonSchema s = (JsonSchema) o; + return getKeyType().equals(s.getKeyType()) && getValueType().equals(s.getValueType()); + } + + @Override + public String toString() { + return key.toString() + " => " + value.toString(); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/VoldemortUtils.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/VoldemortUtils.java new file mode 100644 index 0000000000..db45a503ff --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/VoldemortUtils.java @@ -0,0 +1,131 @@ +package voldemort.store.readonly.mr.utils; + +import java.io.StringReader; +import java.util.List; + +import com.google.common.collect.Lists; + +import voldemort.store.StoreDefinition; +import voldemort.utils.Utils; +import voldemort.xml.StoreDefinitionsMapper; + +public class VoldemortUtils +{ + + /** + * Given the comma separated list of properties as a string, splits it multiple strings + * + * @param paramValue + * Concatenated string + * @param type + * Type of parameter ( to throw exception ) + * @return List of string properties + */ + public static List getCommaSeparatedStringValues(String paramValue, String type) + { + List commaSeparatedProps = Lists.newArrayList(); + for (String url : Utils.COMMA_SEP.split(paramValue.trim())) + if (url.trim().length() > 0) + commaSeparatedProps.add(url); + + if (commaSeparatedProps.size() == 0) + { + throw new RuntimeException("Number of " + type + " should be greater than zero"); + } + return commaSeparatedProps; + } + + public static String getStoreDefXml(String storeName, + int replicationFactor, + int requiredReads, + int requiredWrites, + Integer preferredReads, + Integer preferredWrites, + String keySerializer, + String valSerializer, + String description, + String owners) + { + StringBuffer storeXml = new StringBuffer(); + + storeXml.append("\n\t"); + storeXml.append(storeName); + storeXml.append("\n\tread-only\n\t"); + if (description.length() != 0) + { + storeXml.append(""); + storeXml.append(description); + storeXml.append("\n\t"); + } + if (owners.length() != 0) + { + storeXml.append(""); + storeXml.append(owners); + storeXml.append("\n\t"); + } + storeXml.append("client\n\t"); + storeXml.append(replicationFactor); + storeXml.append("\n\t"); + storeXml.append(requiredReads); + storeXml.append("\n\t"); + storeXml.append(requiredWrites); + storeXml.append("\n\t"); + if (preferredReads != null) + storeXml.append("") + .append(preferredReads) + .append("\n\t"); + if (preferredWrites != null) + storeXml.append("") + .append(preferredWrites) + .append("\n\t"); + storeXml.append(""); + storeXml.append(keySerializer); + storeXml.append("\n\t"); + storeXml.append(valSerializer); + storeXml.append("\n"); + + return storeXml.toString(); + } + + public static String getStoreDefXml(String storeName, + int replicationFactor, + int requiredReads, + int requiredWrites, + Integer preferredReads, + Integer preferredWrites, + String keySerializer, + String valSerializer) + { + StringBuffer storeXml = new StringBuffer(); + + storeXml.append("\n\t"); + storeXml.append(storeName); + storeXml.append("\n\tread-only\n\tclient\n\t"); + storeXml.append(replicationFactor); + storeXml.append("\n\t"); + storeXml.append(requiredReads); + storeXml.append("\n\t"); + storeXml.append(requiredWrites); + storeXml.append("\n\t"); + if (preferredReads != null) + storeXml.append("") + .append(preferredReads) + .append("\n\t"); + if (preferredWrites != null) + storeXml.append("") + .append(preferredWrites) + .append("\n\t"); + storeXml.append(""); + storeXml.append(keySerializer); + storeXml.append("\n\t"); + storeXml.append(valSerializer); + storeXml.append("\n"); + + return storeXml.toString(); + } + + public static StoreDefinition getStoreDef(String xml) + { + return new StoreDefinitionsMapper().readStore(new StringReader(xml)); + } +} diff --git a/lib/azkaban-common-0.05.jar b/lib/azkaban-common-0.05.jar new file mode 100644 index 0000000000..dc52493851 Binary files /dev/null and b/lib/azkaban-common-0.05.jar differ diff --git a/lib/joda-time-1.6.jar b/lib/joda-time-1.6.jar new file mode 100644 index 0000000000..68068a4bcc Binary files /dev/null and b/lib/joda-time-1.6.jar differ diff --git a/lib/mail-1.4.1.jar b/lib/mail-1.4.1.jar new file mode 100644 index 0000000000..59543774f2 Binary files /dev/null and b/lib/mail-1.4.1.jar differ