|
@@ -27,6 +27,7 @@ import junit.extensions.TestSetup;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.io.IntWritable;
|
|
|
import org.apache.hadoop.io.NullWritable;
|
|
@@ -41,6 +42,7 @@ 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.IdentityMapper;
|
|
|
import org.apache.hadoop.mapred.lib.IdentityReducer;
|
|
|
|
|
@@ -247,6 +249,97 @@ public class TestDatamerge extends TestCase {
|
|
|
joinAs("override", OverrideChecker.class);
|
|
|
}
|
|
|
|
|
|
+ public void testNestedJoin() throws Exception {
|
|
|
+ // outer(inner(S1,...,Sn),outer(S1,...Sn))
|
|
|
+ final int SOURCES = 3;
|
|
|
+ final int ITEMS = (SOURCES + 1) * (SOURCES + 1);
|
|
|
+ JobConf job = new JobConf();
|
|
|
+ Path base = cluster.getFileSystem().makeQualified(new Path("/nested"));
|
|
|
+ int[][] source = new int[SOURCES][];
|
|
|
+ for (int i = 0; i < SOURCES; ++i) {
|
|
|
+ source[i] = new int[ITEMS];
|
|
|
+ for (int j = 0; j < ITEMS; ++j) {
|
|
|
+ source[i][j] = (i + 2) * (j + 1);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Path[] src = new Path[SOURCES];
|
|
|
+ SequenceFile.Writer out[] = createWriters(base, job, SOURCES, src);
|
|
|
+ IntWritable k = new IntWritable();
|
|
|
+ for (int i = 0; i < SOURCES; ++i) {
|
|
|
+ IntWritable v = new IntWritable();
|
|
|
+ v.set(i);
|
|
|
+ for (int j = 0; j < ITEMS; ++j) {
|
|
|
+ k.set(source[i][j]);
|
|
|
+ out[i].append(k, v);
|
|
|
+ }
|
|
|
+ out[i].close();
|
|
|
+ }
|
|
|
+ out = null;
|
|
|
+
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append("outer(inner(");
|
|
|
+ for (int i = 0; i < SOURCES; ++i) {
|
|
|
+ sb.append(
|
|
|
+ CompositeInputFormat.compose(SequenceFileInputFormat.class,
|
|
|
+ src[i].toString()));
|
|
|
+ if (i + 1 != SOURCES) sb.append(",");
|
|
|
+ }
|
|
|
+ sb.append("),outer(");
|
|
|
+ sb.append(CompositeInputFormat.compose(FakeIF.class,"foobar"));
|
|
|
+ sb.append(",");
|
|
|
+ for (int i = 0; i < SOURCES; ++i) {
|
|
|
+ sb.append(
|
|
|
+ CompositeInputFormat.compose(SequenceFileInputFormat.class,
|
|
|
+ src[i].toString()));
|
|
|
+ sb.append(",");
|
|
|
+ }
|
|
|
+ sb.append(CompositeInputFormat.compose(FakeIF.class,"raboof") + "))");
|
|
|
+ job.set("mapred.join.expr", sb.toString());
|
|
|
+ job.setInputFormat(CompositeInputFormat.class);
|
|
|
+ Path outf = new Path(base, "out");
|
|
|
+ FileOutputFormat.setOutputPath(job, outf);
|
|
|
+ FakeIF.setKeyClass(job, IntWritable.class);
|
|
|
+ FakeIF.setValClass(job, IntWritable.class);
|
|
|
+
|
|
|
+ job.setMapperClass(IdentityMapper.class);
|
|
|
+ job.setReducerClass(IdentityReducer.class);
|
|
|
+ job.setNumReduceTasks(0);
|
|
|
+ job.setOutputKeyClass(IntWritable.class);
|
|
|
+ job.setOutputValueClass(TupleWritable.class);
|
|
|
+ job.setOutputFormat(SequenceFileOutputFormat.class);
|
|
|
+ JobClient.runJob(job);
|
|
|
+
|
|
|
+ FileStatus[] outlist = cluster.getFileSystem().listStatus(outf);
|
|
|
+ assertEquals(1, outlist.length);
|
|
|
+ assertTrue(0 < outlist[0].getLen());
|
|
|
+ SequenceFile.Reader r =
|
|
|
+ new SequenceFile.Reader(cluster.getFileSystem(),
|
|
|
+ outlist[0].getPath(), job);
|
|
|
+ TupleWritable v = new TupleWritable();
|
|
|
+ while (r.next(k, v)) {
|
|
|
+ assertFalse(((TupleWritable)v.get(1)).has(0));
|
|
|
+ assertFalse(((TupleWritable)v.get(1)).has(SOURCES + 1));
|
|
|
+ boolean chk = true;
|
|
|
+ int ki = k.get();
|
|
|
+ for (int i = 2; i < SOURCES + 2; ++i) {
|
|
|
+ if ((ki % i) == 0 && ki <= i * ITEMS) {
|
|
|
+ assertEquals(i - 2, ((IntWritable)
|
|
|
+ ((TupleWritable)v.get(1)).get((i - 1))).get());
|
|
|
+ } else chk = false;
|
|
|
+ }
|
|
|
+ if (chk) { // present in all sources; chk inner
|
|
|
+ assertTrue(v.has(0));
|
|
|
+ for (int i = 0; i < SOURCES; ++i)
|
|
|
+ assertTrue(((TupleWritable)v.get(0)).has(i));
|
|
|
+ } else { // should not be present in inner join
|
|
|
+ assertFalse(v.has(0));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ r.close();
|
|
|
+ base.getFileSystem(job).delete(base, true);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
public void testConfiguredInputFormat() throws Exception {
|
|
|
JobConf conf = new JobConf();
|
|
|
conf.set("mapred.join.expr", CompositeInputFormat.compose(
|