diff --git a/external/storm-hdfs/README.md b/external/storm-hdfs/README.md index c7ab7ca4bde..65ff3bfe030 100644 --- a/external/storm-hdfs/README.md +++ b/external/storm-hdfs/README.md @@ -553,6 +553,7 @@ Only methods mentioned in **bold** are required. | .setMaxOutstanding() |~~hdfsspout.max.outstanding~~ | 10000 | Limits the number of unACKed tuples by pausing tuple generation (if ACKers are used in the topology) | | .setLockTimeoutSec() |~~hdfsspout.lock.timeout.sec~~ | 5 minutes | Duration of inactivity after which a lock file is considered to be abandoned and ready for another spout to take ownership | | .setClocksInSync() |~~hdfsspout.clocks.insync~~ | true | Indicates whether clocks on the storm machines are in sync (using services like NTP). Used for detecting stale locks. | +| .setInotifyEnabled() |~~hdfsspout.inotify.enabled~~ | false | Enables inotify based watch dir monitoring if set to "true". Note: **root privileges are needed** on HDFS to use inotify | | .withConfigKey() | | | Optional setting. Overrides the default key name ('hdfs.config', see below) used for specifying HDFS client configs. | | .setHdfsClientSettings() |~~hdfs.config~~ (unless changed via withConfigKey)| | Set it to a Map of Key/value pairs indicating the HDFS settings to be used. For example, keytab and principle could be set using this. See section **Using keytabs on all worker hosts** under HDFS bolt below.| | .withOutputStream() | | | Name of output stream. If set, the tuples will be emited to the specified stream. Else tuples will be emited to the default output stream | diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/HdfsDirectoryMonitor.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/HdfsDirectoryMonitor.java new file mode 100644 index 00000000000..1685bc08b70 --- /dev/null +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/HdfsDirectoryMonitor.java @@ -0,0 +1,190 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.storm.hdfs.common; + + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.DFSInotifyEventInputStream; +import org.apache.hadoop.hdfs.client.HdfsAdmin; +import org.apache.hadoop.hdfs.inotify.Event; +import org.apache.hadoop.hdfs.inotify.EventBatch; +import org.apache.hadoop.hdfs.inotify.MissingEventsException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; + +public class HdfsDirectoryMonitor implements Iterable { + public enum WatchMode { POLL, INOTIFY } + + private static final Logger LOG = LoggerFactory.getLogger(HdfsUtils.class); + + private Path watchDir = null; + private WatchMode watchMode = null; + private FileSystem hdfs = null; + private DFSClient dfsClient = null; + private HdfsAdmin hdfsAdmin = null; + + private ArrayList newFiles = new ArrayList<>(); + private long lastTxId = -1; + + private HdfsDirectoryMonitor(Path watchDir, WatchMode watchMode, FileSystem hdfs, DFSClient dfsClient, + HdfsAdmin hdfsAdmin) { + this.watchDir = watchDir; + this.watchMode = watchMode; + this.hdfs = hdfs; + this.dfsClient = dfsClient; + this.hdfsAdmin = hdfsAdmin; + } + + public static HdfsDirectoryMonitor createPollMonitor(Path watchDir, FileSystem hdfs) { + LOG.info("Instantiating Poll based HDFS Directory Monitor"); + return new HdfsDirectoryMonitor(watchDir, WatchMode.POLL, hdfs, null, null); + } + + public static HdfsDirectoryMonitor createInotifyMonitor(Path watchDir, FileSystem hdfs, DFSClient dfsClient, + HdfsAdmin hdfsAdmin) throws IOException { + LOG.info("Instantiating inotify based HDFS Directory Monitor"); + + /* Initialize the last Tx Id to catch the inotify events after the monitor has been created */ + HdfsDirectoryMonitor monitor = new HdfsDirectoryMonitor(watchDir, WatchMode.INOTIFY, hdfs, dfsClient, hdfsAdmin); + monitor.lastTxId = dfsClient.getNamenode().getCurrentEditLogTxid(); + + /* Do an initial collection in the input dir to pick up the leftover files */ + monitor.newFiles = HdfsUtils.listFilesByModificationTime(hdfs, watchDir, 0); + + return monitor; + } + + + public void update() throws IOException, MissingEventsException, InterruptedException { + /* The poll method lists the directory contents while inotify uses incremental updates. + Therefore the new file list needs to be replaced in case of poll but added to existing in case of inotify. + */ + if (this.watchMode == WatchMode.POLL) { + newFiles = HdfsUtils.listFilesByModificationTime(hdfs, watchDir, 0); + } else { + newFiles.addAll(getNewFilesFromInotify(watchDir.toString())); + } + } + + private boolean filterFiles(String watchDirectory, String filename) + { + /* include files which are located inside the watch directory */ + return !filename.equals(watchDirectory) && + filename.startsWith(watchDirectory.endsWith("/") ? watchDirectory : watchDirectory + "/"); + } + + private ArrayList getNewFilesFromInotify(String watchDirectory) + throws IOException, InterruptedException, MissingEventsException { + + if (lastTxId == -1) { + lastTxId = dfsClient.getNamenode().getCurrentEditLogTxid(); + } + + LOG.trace("Reading inotify events from HDFS from tx id: " + lastTxId); + + DFSInotifyEventInputStream eventStream = hdfsAdmin.getInotifyEventStream(this.lastTxId); + + EventBatch batch; + Set modifiedFiles = new HashSet<>(); + while ( (batch = eventStream.poll()) != null) { + for (Event event : batch.getEvents()) { + switch (event.getEventType()) { + case CREATE: { + /* inotify event's getPath() returns a string, not a real Path object. + We need to construct the full Path object using hdfs.makeQualified() + to get the same granularity (scheme + authority + path) as created + by HdfsUtils.listFilesByModificationTime */ + String filename = ((Event.CreateEvent) event).getPath(); + + if (filterFiles(watchDirectory, filename)) { + Path path = hdfs.makeQualified(new Path(filename)); + modifiedFiles.add(path); + } + } + break; + + case UNLINK: { + String filename = ((Event.UnlinkEvent) event).getPath(); + + if (filterFiles(watchDirectory, filename)) { + Path path = hdfs.makeQualified(new Path(filename)); + modifiedFiles.add(path); + } + } + break; + + case RENAME: { + String srcFilename = ((Event.RenameEvent) event).getSrcPath(); + String dstFilename = ((Event.RenameEvent) event).getDstPath(); + + if (filterFiles(watchDirectory, srcFilename) && filterFiles(watchDirectory, dstFilename)) { + Path srcPath = hdfs.makeQualified(new Path(srcFilename)); + Path dstPath = hdfs.makeQualified(new Path(dstFilename)); + + modifiedFiles.remove(srcPath); + modifiedFiles.add(dstPath); + } + } + break; + } + } + + lastTxId = batch.getTxid(); + } + + return new ArrayList<>(modifiedFiles); + } + + @Override + public Iterator iterator() { + return new HdfsDirectoryMonitorIterator(); + } + + private class HdfsDirectoryMonitorIterator implements Iterator { + HdfsDirectoryMonitorIterator() { + } + + @Override + public boolean hasNext() { + return newFiles.size() > 0; + } + + @Override + public Path next() { + if (newFiles == null || newFiles.size() == 0) { + throw new NoSuchElementException(); + } + return newFiles.remove(0); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Delete Not Supported"); + } + } +} diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/Configs.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/Configs.java index 8911b3c7b85..5de5b723237 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/Configs.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/Configs.java @@ -34,6 +34,7 @@ public class Configs { public static final String LOCK_TIMEOUT = "hdfsspout.lock.timeout.sec"; // inactivity duration after which locks are considered candidates for being reassigned to another spout public static final String CLOCKS_INSYNC = "hdfsspout.clocks.insync"; // if clocks on machines in the Storm cluster are in sync public static final String IGNORE_SUFFIX = "hdfsspout.ignore.suffix"; // filenames with this suffix in archive dir will be ignored by the Spout + public static final String INOTIFY_ENABLED = "hdfsspout.inotify.enabled"; // enables inotify based directory watch mode if set to "true" public static final String DEFAULT_LOCK_DIR = ".lock"; public static final int DEFAULT_COMMIT_FREQ_COUNT = 20000; diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java index b7627f24178..4d597ae70d0 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.net.URI; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -30,11 +29,14 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.client.HdfsAdmin; import org.apache.storm.Config; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.storm.hdfs.common.HdfsUtils; +import org.apache.storm.hdfs.common.HdfsDirectoryMonitor; import org.apache.storm.hdfs.common.security.HdfsSecurityUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,6 +84,8 @@ public class HdfsSpout extends BaseRichSpout { private FileSystem hdfs; private FileReader reader; + private HdfsDirectoryMonitor hdfsDirectoryMonitor; + private boolean inotifyEnabled = false; private SpoutOutputCollector collector; HashMap > inflight = new HashMap<>(); @@ -169,6 +173,11 @@ public HdfsSpout setIgnoreSuffix(String ignoreSuffix) { return this; } + public HdfsSpout setInotifyEnabled(boolean inotifyEnabled) { + this.inotifyEnabled = inotifyEnabled; + return this; + } + /** Output field names. Number of fields depends upon the reader type */ public HdfsSpout withOutputFields(String... fields) { outputFields = new Fields(fields); @@ -503,6 +512,23 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect clocksInSync = Boolean.parseBoolean(conf.get(Configs.CLOCKS_INSYNC).toString()); } + if ( conf.get(Configs.INOTIFY_ENABLED) != null ) { + this.inotifyEnabled = Boolean.parseBoolean(conf.get(Configs.INOTIFY_ENABLED).toString()); + } + + if (this.inotifyEnabled) { + try { + DFSClient dfsClient = new DFSClient(URI.create(hdfsUri), hdfsConfig); + HdfsAdmin hdfsAdmin = new HdfsAdmin(hdfs.getUri(), hdfsConfig); + this.hdfsDirectoryMonitor = HdfsDirectoryMonitor.createInotifyMonitor(sourceDirPath, hdfs, dfsClient, hdfsAdmin); + } catch (IOException e) { + LOG.error("Error creating inotify based HDFS directory monitor. Are we running as root?", e); + throw new RuntimeException("Error creating inotify based HDFS directory monitor. Are we running as root?", e); + } + } else { + this.hdfsDirectoryMonitor = HdfsDirectoryMonitor.createPollMonitor(sourceDirPath, hdfs); + } + // -- spout id spoutId = context.getThisComponentId(); @@ -596,13 +622,16 @@ private FileReader pickNextFile() { } // 2) If no abandoned files, then pick oldest file in sourceDirPath, lock it and rename it - Collection listing = HdfsUtils.listFilesByModificationTime(hdfs, sourceDirPath, 0); - - for (Path file : listing) { - if (file.getName().endsWith(inprogress_suffix)) { - continue; - } - if (file.getName().endsWith(ignoreSuffix)) { + try { + hdfsDirectoryMonitor.update(); + } catch (Exception e) { + LOG.error("Unable to update source dir: " + sourceDirPath, e); + return null; + } + for (Path file : hdfsDirectoryMonitor) { + if (Path.getPathWithoutSchemeAndAuthority(file).toString().startsWith(lockDirPath.toString()) || + file.getName().endsWith(inprogress_suffix) || + file.getName().endsWith(ignoreSuffix)) { continue; } lock = FileLock.tryLock(hdfs, file, lockDirPath, spoutId); diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestHdfsSpout.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestHdfsSpout.java index f60cbf3e315..985e59ab83e 100644 --- a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestHdfsSpout.java +++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestHdfsSpout.java @@ -44,6 +44,10 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; import java.io.BufferedReader; import java.io.File; @@ -52,6 +56,7 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,6 +64,7 @@ import org.apache.storm.hdfs.common.HdfsUtils.Pair; +@RunWith(Parameterized.class) public class TestHdfsSpout { @Rule @@ -72,6 +78,14 @@ public class TestHdfsSpout { public TestHdfsSpout() { } + @Parameter + public String inotifyEnabled; + + @Parameters(name="inotifyEnabled={0}") + public static Collection inotifyEnabledValues() { + return Arrays.asList(new Object[][] { { "false" }, { "true" } }); + } + static MiniDFSCluster.Builder builder; static MiniDFSCluster hdfsCluster; static DistributedFileSystem fs; @@ -111,12 +125,6 @@ public void shutDown() throws IOException { @Test public void testSimpleText_noACK() throws IOException { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 5); - - Path file2 = new Path(source.toString() + "/file2.txt"); - createTextFile(file2, 5); - HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); spout.setCommitFrequencyCount(1); spout.setCommitFrequencySec(1); @@ -124,6 +132,12 @@ public void testSimpleText_noACK() throws IOException { Map conf = getCommonConfigs(); openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 5); + + Path file2 = new Path(source.toString() + "/file2.txt"); + createTextFile(file2, 5); + runSpout(spout,"r11"); Path arc1 = new Path(archive.toString() + "/file1.txt"); @@ -133,12 +147,6 @@ public void testSimpleText_noACK() throws IOException { @Test public void testSimpleText_ACK() throws IOException { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 5); - - Path file2 = new Path(source.toString() + "/file2.txt"); - createTextFile(file2, 5); - HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); spout.setCommitFrequencyCount(1); spout.setCommitFrequencySec(1); @@ -147,6 +155,12 @@ public void testSimpleText_ACK() throws IOException { conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, "1"); // enable ACKing openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 5); + + Path file2 = new Path(source.toString() + "/file2.txt"); + createTextFile(file2, 5); + // consume file 1 runSpout(spout, "r6", "a0", "a1", "a2", "a3", "a4"); Path arc1 = new Path(archive.toString() + "/file1.txt"); @@ -160,9 +174,6 @@ public void testSimpleText_ACK() throws IOException { @Test public void testResumeAbandoned_Text_NoAck() throws Exception { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 6); - final Integer lockExpirySec = 1; HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); @@ -180,6 +191,9 @@ public void testResumeAbandoned_Text_NoAck() throws Exception { openSpout(spout, 0, conf); openSpout(spout2, 1, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 6); + // consume file 1 partially List res = runSpout(spout, "r2"); Assert.assertEquals(2, res.size()); @@ -217,9 +231,6 @@ public void testResumeAbandoned_Text_NoAck() throws Exception { @Test public void testResumeAbandoned_Seq_NoAck() throws Exception { - Path file1 = new Path(source.toString() + "/file1.seq"); - createSeqFile(fs, file1, 6); - final Integer lockExpirySec = 1; HdfsSpout spout = makeSpout(Configs.SEQ, SequenceFileReader.defaultFields); @@ -237,6 +248,9 @@ public void testResumeAbandoned_Seq_NoAck() throws Exception { openSpout(spout, 0, conf); openSpout(spout2, 1, conf); + Path file1 = new Path(source.toString() + "/file1.seq"); + createSeqFile(fs, file1, 6); + // consume file 1 partially List res = runSpout(spout, "r2"); Assert.assertEquals(2, res.size()); @@ -340,9 +354,6 @@ private List listDir(Path p) throws IOException { @Test public void testMultipleFileConsumption_Ack() throws Exception { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 5); - HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); spout.setCommitFrequencyCount(1); @@ -352,6 +363,9 @@ public void testMultipleFileConsumption_Ack() throws Exception { conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, "1"); // enable ACKing openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 5); + // read few lines from file1 dont ack runSpout(spout, "r3"); FileReader reader = getField(spout, "reader"); @@ -411,25 +425,44 @@ private static boolean getBoolField(HdfsSpout spout, String fieldName) throws No } + @Test + public void testLeftoverFiles() throws IOException { + Path file1 = new Path(source.toString() + "/leftover1.txt"); + createTextFile(file1, 5); + + Path file2 = new Path(source.toString() + "/leftover2.txt"); + createTextFile(file2, 5); + + HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); + Map conf = getCommonConfigs(); + openSpout(spout, 0, conf); + + runSpout(spout,"r11"); + + List res = runSpout(spout, "r11"); + Assert.assertEquals(10, res.size()); + + Assert.assertEquals(2, listDir(archive).size()); + Assert.assertEquals(0, listDir(source).size()); + } + @Test public void testSimpleSequenceFile() throws IOException { - //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); + HdfsSpout spout = makeSpout(Configs.SEQ, SequenceFileReader.defaultFields); + Map conf = getCommonConfigs(); + openSpout(spout, 0, conf); + Path file1 = new Path(source + "/file1.seq"); createSeqFile(fs, file1, 5); Path file2 = new Path(source + "/file2.seq"); createSeqFile(fs, file2, 5); - - HdfsSpout spout = makeSpout(Configs.SEQ, SequenceFileReader.defaultFields); - Map conf = getCommonConfigs(); - openSpout(spout, 0, conf); - // consume both files List res = runSpout(spout, "r11"); Assert.assertEquals(10, res.size()); @@ -445,7 +478,10 @@ public void testSimpleSequenceFile() throws IOException { @Test public void testReadFailures() throws Exception { - // 1) create couple of input files to read + HdfsSpout spout = makeSpout(MockTextFailingReader.class.getName(), MockTextFailingReader.defaultFields); + Map conf = getCommonConfigs(); + openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); Path file2 = new Path(source.toString() + "/file2.txt"); @@ -453,16 +489,11 @@ public void testReadFailures() throws Exception { createTextFile(file2, 7); Assert.assertEquals(2, listDir(source).size()); - // 2) run spout - HdfsSpout spout = makeSpout(MockTextFailingReader.class.getName(), MockTextFailingReader.defaultFields); - Map conf = getCommonConfigs(); - openSpout(spout, 0, conf); - List 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 + // make sure 6 lines (3 from each file) were read in all Assert.assertEquals(((MockCollector) spout.getCollector()).lines.size(), 6); ArrayList badFiles = HdfsUtils.listFilesByModificationTime(fs, badfiles, 0); Assert.assertEquals(badFiles.size(), 2); @@ -471,19 +502,17 @@ public void testReadFailures() throws Exception { // check lock creation/deletion and contents @Test public void testLocking() throws Exception { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 10); - // 0) config spout to log progress in lock file for each tuple - HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); spout.setCommitFrequencyCount(1); spout.setCommitFrequencySec(1000); // effectively disable commits based on time - Map conf = getCommonConfigs(); openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 10); + // 1) read initial lines in file, then check if lock exists List res = runSpout(spout, "r5"); Assert.assertEquals(5, res.size()); @@ -522,9 +551,6 @@ public void testLocking() throws Exception { @Test public void testLockLoggingFreqCount() throws Exception { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 10); - // 0) config spout to log progress in lock file for each tuple HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); @@ -534,6 +560,9 @@ public void testLockLoggingFreqCount() throws Exception { Map conf = getCommonConfigs(); openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 10); + // 1) read 5 lines in file, runSpout(spout, "r5"); @@ -550,9 +579,6 @@ public void testLockLoggingFreqCount() throws Exception { @Test public void testLockLoggingFreqSec() throws Exception { - Path file1 = new Path(source.toString() + "/file1.txt"); - createTextFile(file1, 10); - // 0) config spout to log progress in lock file for each tuple HdfsSpout spout = makeSpout(Configs.TEXT, TextFileReader.defaultFields); spout.setCommitFrequencyCount(0); // disable it @@ -561,6 +587,9 @@ public void testLockLoggingFreqSec() throws Exception { Map conf = getCommonConfigs(); openSpout(spout, 0, conf); + Path file1 = new Path(source.toString() + "/file1.txt"); + createTextFile(file1, 10); + // 1) read 5 lines in file runSpout(spout, "r5"); @@ -590,6 +619,7 @@ private static List readTextFile(FileSystem fs, String f) throws IOExcep private Map getCommonConfigs() { Map conf = new HashMap(); + conf.put(Configs.INOTIFY_ENABLED, inotifyEnabled); conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, "0"); return conf; } diff --git a/pom.xml b/pom.xml index 7f4f965c6f3..910ded3c5de 100644 --- a/pom.xml +++ b/pom.xml @@ -234,7 +234,7 @@ 0.2.4 3.3.2 0.9.0 - 16.0.1 + 14.0.1 3.9.0.Final 1.6.6 2.1 @@ -248,7 +248,7 @@ 0.1.0 1.2.0 0.14.0 - 2.6.1 + 2.7.3 1.1.0 3.0.3 2.5