/** * Execute a sequence of calls on HdfsSpout. * * @param cmds: set of commands to run, e.g. "r,r,r,r,a1,f2,...". The commands are: r[N] - receive() called N times aN - ack, item * number: N fN - fail, item number: N */ private List<String> runSpout(HdfsSpout spout, String... cmds) { MockCollector collector = (MockCollector) spout.getCollector(); for (String cmd : cmds) { if (cmd.startsWith("r")) { int count = 1; if (cmd.length() > 1) { count = Integer.parseInt(cmd.substring(1)); } for (int i = 0; i < count; ++i) { spout.nextTuple(); } } else if (cmd.startsWith("a")) { int n = Integer.parseInt(cmd.substring(1)); Pair<HdfsSpout.MessageId, List<Object>> item = collector.items.get(n); spout.ack(item.getKey()); } else if (cmd.startsWith("f")) { int n = Integer.parseInt(cmd.substring(1)); Pair<HdfsSpout.MessageId, List<Object>> item = collector.items.get(n); spout.fail(item.getKey()); } } return collector.lines; }
@Test public void testSimpleText_noACK() throws Exception { Path file1 = new Path(source.toString() + "/file1.txt"); createTextFile(file1, 5); Path file2 = new Path(source.toString() + "/file2.txt"); createTextFile(file2, 5); try (AutoCloseableHdfsSpout closeableSpout = makeSpout(Configs.TEXT, TextFileReader.defaultFields)) { HdfsSpout spout = closeableSpout.spout; spout.setCommitFrequencyCount(1); spout.setCommitFrequencySec(1); Map<String, Object> conf = getCommonConfigs(); openSpout(spout, 0, conf); runSpout(spout, "r11"); Path arc1 = new Path(archive.toString() + "/file1.txt"); Path arc2 = new Path(archive.toString() + "/file2.txt"); checkCollectorOutput_txt((MockCollector) spout.getCollector(), arc1, arc2); } }
@Test public void testEmptySimpleText_ACK() throws Exception { Path file1 = new Path(source.toString() + "/file_empty.txt"); createTextFile(file1, 0); Path file2 = new Path(source.toString() + "/file.txt"); createTextFile(file2, 5); try (AutoCloseableHdfsSpout closeableSpout = makeSpout(Configs.TEXT, TextFileReader.defaultFields)) { HdfsSpout spout = closeableSpout.spout; spout.setCommitFrequencyCount(1); Map<String, Object> conf = getCommonConfigs(); conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, "1"); // enable ACKing openSpout(spout, 0, conf); // consume empty file runSpout(spout, "r1"); Path arc1 = new Path(archive.toString() + "/file_empty.txt"); checkCollectorOutput_txt((MockCollector) spout.getCollector(), arc1); // consume file 2 runSpout(spout, "r5", "a0", "a1", "a2", "a3", "a4"); Path arc2 = new Path(archive.toString() + "/file.txt"); checkCollectorOutput_txt((MockCollector) spout.getCollector(), arc1, arc2); } }
@Test public void testSimpleText_ACK() throws Exception { Path file1 = new Path(source.toString() + "/file1.txt"); createTextFile(file1, 5); Path file2 = new Path(source.toString() + "/file2.txt"); createTextFile(file2, 5); try (AutoCloseableHdfsSpout closeableSpout = makeSpout(Configs.TEXT, TextFileReader.defaultFields)) { HdfsSpout spout = closeableSpout.spout; spout.setCommitFrequencyCount(1); spout.setCommitFrequencySec(1); Map<String, Object> conf = getCommonConfigs(); conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, "1"); // enable ACKing openSpout(spout, 0, conf); // consume file 1 runSpout(spout, "r6", "a0", "a1", "a2", "a3", "a4"); Path arc1 = new Path(archive.toString() + "/file1.txt"); checkCollectorOutput_txt((MockCollector) spout.getCollector(), arc1); // consume file 2 runSpout(spout, "r6", "a5", "a6", "a7", "a8", "a9"); Path arc2 = new Path(archive.toString() + "/file2.txt"); checkCollectorOutput_txt((MockCollector) spout.getCollector(), arc1, arc2); } }
@Test public void testReadFailures() throws Exception { // 1) create couple of input files to read Path file1 = new Path(source.toString() + "/file1.txt"); Path file2 = new Path(source.toString() + "/file2.txt"); createTextFile(file1, 6); createTextFile(file2, 7); Assert.assertEquals(2, listDir(source).size()); // 2) run spout try ( AutoCloseableHdfsSpout closeableSpout = makeSpout(MockTextFailingReader.class.getName(), MockTextFailingReader.defaultFields)) { HdfsSpout spout = closeableSpout.spout; Map<String, Object> conf = getCommonConfigs(); openSpout(spout, 0, conf); List<String> res = runSpout(spout, "r11"); String[] expected = new String[]{ "[line 0]", "[line 1]", "[line 2]", "[line 0]", "[line 1]", "[line 2]" }; Assert.assertArrayEquals(expected, res.toArray()); // 3) make sure 6 lines (3 from each file) were read in all Assert.assertEquals(((MockCollector) spout.getCollector()).lines.size(), 6); ArrayList<Path> badFiles = HdfsUtils.listFilesByModificationTime(fs, badfiles, 0); Assert.assertEquals(badFiles.size(), 2); } }
@Test public void testSimpleSequenceFile() throws Exception { //1) create a couple files to consume source = new Path("/tmp/hdfsspout/source"); fs.mkdirs(source); archive = new Path("/tmp/hdfsspout/archive"); fs.mkdirs(archive); Path file1 = new Path(source + "/file1.seq"); createSeqFile(fs, file1, 5); Path file2 = new Path(source + "/file2.seq"); createSeqFile(fs, file2, 5); try (AutoCloseableHdfsSpout closeableSpout = makeSpout(Configs.SEQ, SequenceFileReader.defaultFields)) { HdfsSpout spout = closeableSpout.spout; Map<String, Object> conf = getCommonConfigs(); openSpout(spout, 0, conf); // consume both files List<String> res = runSpout(spout, "r11"); Assert.assertEquals(10, res.size()); Assert.assertEquals(2, listDir(archive).size()); Path f1 = new Path(archive + "/file1.seq"); Path f2 = new Path(archive + "/file2.seq"); checkCollectorOutput_seq((MockCollector) spout.getCollector(), f1, f2); } }