}
private DAG createDag(TezConfiguration tezConf, Path streamPath, Path hashPath, Path outPath,
int numPartitions, boolean doBroadcast) throws IOException {
DAG dag = DAG.create("JoinExample" + (doBroadcast ? "-WithBroadcast" : ""));
/**
* This vertex represents the side of the join that will be accumulated in a hash
* table in order to join it against the other side. It reads text data using the
* TextInputFormat. ForwardingProcessor simply forwards the data downstream as is.
*/
Vertex hashFileVertex = Vertex.create(hashSide, ProcessorDescriptor.create(
ForwardingProcessor.class.getName())).addDataSource(
inputFile,
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
hashPath.toUri().toString()).groupSplits(false).build());
/**
* This vertex represents that side of the data that will be streamed and joined
* against the other side that has been accumulated into a hash table. It reads
* text data using the TextInputFormat. ForwardingProcessor simply forwards the data
* downstream as is.
*/
Vertex streamFileVertex = Vertex.create(streamingSide, ProcessorDescriptor.create(
ForwardingProcessor.class.getName())).addDataSource(
inputFile,
MRInput
.createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
streamPath.toUri().toString()).groupSplits(false).build());
/**
* This vertex represents the join operation. It writes the join output as text using
* the TextOutputFormat. The JoinProcessor is going to perform the join of the
* streaming side and the hash side. It is load balanced across numPartitions
*/
Vertex joinVertex = Vertex.create(joiner, ProcessorDescriptor.create(
JoinProcessor.class.getName()), numPartitions).addDataSink(joinOutput,
MROutput.createConfigBuilder(new Configuration(tezConf),
TextOutputFormat.class, outPath.toUri().toString()).build());
/**
* The streamed side will be partitioned into fragments with the same keys going to
* the same fragments using hash partitioning. The data to be joined is the key itself
* and so the value is null. The number of fragments is initially inferred from the
* number of tasks running in the join vertex because each task will be handling one
* fragment.
*/
UnorderedPartitionedKVEdgeConfig streamConf =
UnorderedPartitionedKVEdgeConfig
.newBuilder(Text.class.getName(), NullWritable.class.getName(),
HashPartitioner.class.getName()).build();
/**
* Connect the join vertex with the stream side
*/
Edge e1 = Edge.create(streamFileVertex, joinVertex, streamConf.createDefaultEdgeProperty());
EdgeProperty hashSideEdgeProperty = null;
if (doBroadcast) {
/**
* This option can be used when the hash side is small. We can broadcast the entire data to
* all fragments of the stream side. This avoids re-partitioning the fragments of the stream
* side to match the partitioning scheme of the hash side and avoids costly network data
* transfer. However, in this example the stream side is being partitioned in both cases for
* brevity of code. The join task can perform the join of its fragment of keys with all the
* keys of the hash side.
* Using an unpartitioned edge to transfer the complete output of the hash side to be
* broadcasted to all fragments of the streamed side. Again, since the data is the key, the
* value is null.
*/
UnorderedKVEdgeConfig broadcastConf = UnorderedKVEdgeConfig.newBuilder(Text.class.getName(),
NullWritable.class.getName()).build();
hashSideEdgeProperty = broadcastConf.createDefaultBroadcastEdgeProperty();
} else {
/**
* The hash side is also being partitioned into fragments with the same key going to the same
* fragment using hash partitioning. This way all keys with the same hash value will go to the
* same fragment from both sides. Thus the join task handling that fragment can join both data
* set fragments.
*/
hashSideEdgeProperty = streamConf.createDefaultEdgeProperty();
}
/**
* Connect the join vertex to the hash side.
* The join vertex is connected with 2 upstream vertices that provide it with inputs
*/
Edge e2 = Edge.create(hashFileVertex, joinVertex, hashSideEdgeProperty);
/**
* Connect everything up by adding them to the DAG
*/
dag.addVertex(streamFileVertex).addVertex(hashFileVertex).addVertex(joinVertex)
.addEdge(e1).addEdge(e2);
return dag;
}