Renaming log to LOG. Adding {} around all IF stmts. Tweaking logs and log
levels. Doc updates.
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b5240a73
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b5240a73
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b5240a73
Branch: refs/heads/1.x-branch
Commit: b5240a73bcd56a6883cb46766943e9d76edffc75
Parents: ac1322f
Author: Roshan Naik <[email protected]>
Authored: Tue Jan 5 20:12:52 2016 -0800
Committer: Roshan Naik <[email protected]>
Committed: Thu Jan 14 11:34:56 2016 -0800
----------------------------------------------------------------------
.../jvm/storm/starter/HdfsSpoutTopology.java | 40 ++--
external/storm-hdfs/README.md | 229 ++++++++++---------
.../storm/hdfs/spout/AbstractFileReader.java | 14 +-
.../org/apache/storm/hdfs/spout/DirLock.java | 23 +-
.../org/apache/storm/hdfs/spout/FileLock.java | 55 +++--
.../org/apache/storm/hdfs/spout/HdfsSpout.java | 168 ++++++++------
.../storm/hdfs/spout/ProgressTracker.java | 9 +-
.../storm/hdfs/spout/SequenceFileReader.java | 18 +-
.../apache/storm/hdfs/spout/TextFileReader.java | 24 +-
.../apache/storm/hdfs/spout/TestDirLock.java | 3 +-
.../apache/storm/hdfs/spout/TestFileLock.java | 9 +-
.../apache/storm/hdfs/spout/TestHdfsSpout.java | 9 +-
pom.xml | 2 +-
13 files changed, 330 insertions(+), 273 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java
----------------------------------------------------------------------
diff --git
a/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java
b/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java
index 3837943..ca6b045 100644
--- a/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java
+++ b/examples/storm-starter/src/jvm/storm/starter/HdfsSpoutTopology.java
@@ -24,13 +24,6 @@ import backtype.storm.generated.Nimbus;
import backtype.storm.topology.TopologyBuilder;
import backtype.storm.utils.NimbusClient;
import backtype.storm.utils.Utils;
-import org.apache.storm.hdfs.bolt.HdfsBolt;
-import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
-import org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat;
-import org.apache.storm.hdfs.bolt.format.RecordFormat;
-import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
-import org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy;
-import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
import org.apache.storm.hdfs.spout.Configs;
import org.apache.storm.hdfs.spout.HdfsSpout;
import backtype.storm.topology.base.BaseRichBolt;
@@ -46,10 +39,8 @@ import java.util.Map;
public class HdfsSpoutTopology {
public static final String SPOUT_ID = "hdfsspout";
- public static final String BOLT_ID = "hdfsbolt";
+ public static final String BOLT_ID = "constbolt";
- public static final int SPOUT_NUM = 1;
- public static final int BOLT_NUM = 1;
public static final int WORKER_NUM = 1;
public static class ConstBolt extends BaseRichBolt {
@@ -57,6 +48,7 @@ public class HdfsSpoutTopology {
public static final String FIELDS = "message";
private OutputCollector collector;
private static final Logger log = LoggerFactory.getLogger(ConstBolt.class);
+ int count =0;
public ConstBolt() {
}
@@ -69,7 +61,13 @@ public class HdfsSpoutTopology {
@Override
public void execute(Tuple tuple) {
log.info("Received tuple : {}", tuple.getValue(0));
- collector.ack(tuple);
+ count++;
+ if(count==3) {
+ collector.fail(tuple);
+ }
+ else {
+ collector.ack(tuple);
+ }
}
@Override
@@ -78,9 +76,7 @@ public class HdfsSpoutTopology {
}
} // class
- /** Copies text file content from sourceDir to destinationDir. Moves source
files into sourceDir after its done consuming
- * args: sourceDir sourceArchiveDir badDir destinationDir
- */
+ /** Copies text file content from sourceDir to destinationDir. Moves source
files into sourceDir after its done consuming */
public static void main(String[] args) throws Exception {
// 0 - validate args
if (args.length < 7) {
@@ -91,10 +87,9 @@ public class HdfsSpoutTopology {
System.err.println(" hdfsUri - hdfs name node URI");
System.err.println(" fileFormat - Set to 'TEXT' for reading text files
or 'SEQ' for sequence files.");
System.err.println(" sourceDir - read files from this HDFS dir using
HdfsSpout.");
- System.err.println(" sourceArchiveDir - after a file in sourceDir is
read completely, it is moved to this HDFS location.");
+ System.err.println(" archiveDir - after a file in sourceDir is read
completely, it is moved to this HDFS location.");
System.err.println(" badDir - files that cannot be read properly will be
moved to this HDFS location.");
- System.err.println(" destinationDir - write data out to this HDFS
location using HDFS bolt.");
-
+ System.err.println(" spoutCount - Num of spout instances.");
System.err.println();
System.exit(-1);
}
@@ -106,12 +101,12 @@ public class HdfsSpoutTopology {
String sourceDir = args[3];
String sourceArchiveDir = args[4];
String badDir = args[5];
- String destinationDir = args[6];
+ int spoutNum = Integer.parseInt(args[6]);
// 2 - create and configure spout and bolt
ConstBolt bolt = new ConstBolt();
- HdfsSpout spout = new HdfsSpout().withOutputFields("line");
+ HdfsSpout spout = new HdfsSpout().withOutputFields("line");
Config conf = new Config();
conf.put(Configs.SOURCE_DIR, sourceDir);
conf.put(Configs.ARCHIVE_DIR, sourceArchiveDir);
@@ -120,6 +115,7 @@ public class HdfsSpoutTopology {
conf.put(Configs.HDFS_URI, hdfsUri);
conf.setDebug(true);
conf.setNumWorkers(1);
+ conf.setNumAckers(1);
conf.setMaxTaskParallelism(1);
// 3 - Create and configure topology
@@ -128,10 +124,10 @@ public class HdfsSpoutTopology {
conf.registerMetricsConsumer(backtype.storm.metric.LoggingMetricsConsumer.class);
TopologyBuilder builder = new TopologyBuilder();
- builder.setSpout(SPOUT_ID, spout, SPOUT_NUM);
- builder.setBolt(BOLT_ID, bolt, BOLT_NUM).shuffleGrouping(SPOUT_ID);
+ builder.setSpout(SPOUT_ID, spout, spoutNum);
+ builder.setBolt(BOLT_ID, bolt, 1).shuffleGrouping(SPOUT_ID);
- // 4 - submit topology, wait for few min and terminate it
+ // 4 - submit topology, wait for a few min and terminate it
Map clusterConf = Utils.readStormConfig();
StormSubmitter.submitTopologyWithProgressBar(topologyName, conf,
builder.createTopology());
Nimbus.Client client =
NimbusClient.getConfiguredClient(clusterConf).getClient();
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/README.md
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/README.md b/external/storm-hdfs/README.md
index 237fc8c..8008bd8 100644
--- a/external/storm-hdfs/README.md
+++ b/external/storm-hdfs/README.md
@@ -1,109 +1,10 @@
# Storm HDFS
Storm components for interacting with HDFS file systems
- - HDFS Spout
- HDFS Bolt
+ - HDFS Spout
-# HDFS Spout
-
-Hdfs spout is intended to allow feeding data into Storm from a HDFS directory.
-It will actively monitor the directory to consume any new files that appear in
the directory.
-
-**Impt**: Hdfs spout assumes that the files being made visible to it in the
monitored directory
-are NOT actively being written to. Only after a files is completely written
should it be made
-visible to the spout. This can be achieved by either writing the files out to
another directory
-and once completely written, move it to the monitored directory. Alternatively
the file
-can be created with a '.ignore' suffix in the monitored directory and after
data is completely
-written, rename it without the suffix. File names with a '.ignore' suffix are
ignored
-by the spout.
-
-When the spout is actively consuming a file, ite renames the file with a
'.inprogress' suffix.
-After consuming all the contents in the file, the file will be moved to a
configurable *done*
-directory and the '.inprogress' suffix will be dropped.
-
-**Concurrency** If multiple spout instances are used in the topology, each
instance will consume
-a different file. Synchronization among spout instances is done using a lock
files created in
-(by default) a '.lock' subdirectory under the monitored directory. A file with
the same name
-as the file being consumed (with the in progress suffix) is created in the
lock directory.
-Once the file is completely consumed, the corresponding lock file is deleted.
-
-**Recovery from failure**
-Periodically, the spout also records progress information wrt to how much of
the file has been
-consumed in the lock file. In case of an crash of the spout instance (or force
kill of topology)
-another spout can take over the file and resume from the location recorded in
the lock file.
-
-Certain error conditions can cause lock files to go stale. Basically the lock
file exists,
-but no spout actively owns and therefore will the file will not be deleted.
Usually this indicates
-that the corresponding input file has also not been completely processed. A
configuration
-'hdfsspout.lock.timeout.sec' can be set to specify the duration of inactivity
that a lock file
-should be considered stale. Stale lock files are candidates for automatic
transfer of ownership to
-another spout.
-
-**Lock on .lock Directory**
-The .lock directory contains another DIRLOCK file which is used to co-ordinate
accesses to the
-.lock dir itself among spout instances. A spout will try to create it when it
needs access to
-the .lock directory and then delete it when done. In case of a topology crash
or force kill,
-if this file still exists, it should be deleted to allow the new topology
instance to regain
-full access to the .lock directory and resume normal processing.
-
-## Usage
-
-The following example creates an HDFS spout that reads text files from HDFS
path hdfs://localhost:54310/source.
-
-```java
-// Instantiate spout
-HdfsSpout textReaderSpout = new
HdfsSpout().withOutputFields(TextFileReader.defaultFields);
-// HdfsSpout seqFileReaderSpout = new
HdfsSpout().withOutputFields(SequenceFileReader.defaultFields);
-
-// textReaderSpout.withConfigKey("custom.keyname"); // Optional. Not required
normally unless you need to change the keyname use to provide hds settings.
This keyname defaults to 'hdfs.config'
-
-// Configure it
-Config conf = new Config();
-conf.put(Configs.SOURCE_DIR, "hdfs://localhost:54310/source");
-conf.put(Configs.ARCHIVE_DIR, "hdfs://localhost:54310/done");
-conf.put(Configs.BAD_DIR, "hdfs://localhost:54310/badfiles");
-conf.put(Configs.READER_TYPE, "text"); // or 'seq' for sequence files
-
-// Create & configure topology
-TopologyBuilder builder = new TopologyBuilder();
-builder.setSpout("hdfsspout", textReaderSpout, SPOUT_NUM);
-
-// Setup bolts and other topology configuration
- ..snip..
-
-// Submit topology with config
-StormSubmitter.submitTopologyWithProgressBar("topologyName", conf,
builder.createTopology());
-```
-
-## Configuration Settings
-Class HdfsSpout provided following methods for configuration:
-
-`HdfsSpout withOutputFields(String... fields)` : This sets the names for the
output fields.
-The number of fields depends upon the reader being used. For convenience,
built-in reader types
-expose a static member called `defaultFields` that can be used for this.
-
- `HdfsSpout withConfigKey(String configKey)`
-Allows overriding the default key name (hdfs.config) with new name for
specifying HDFS configs. Typicallly used
-to provide kerberos keytabs.
-
-Only settings mentioned in **bold** are required.
-
-| Setting | Default | Description |
-|------------------------------|-------------|-------------|
-|**hdfsspout.reader.type** | | Indicates the reader for the
file format. Set to 'seq' for reading sequence files or 'text' for text files.
Set to a fully qualified class name if using a custom type (that implements
interface org.apache.storm.hdfs.spout.FileReader)|
-|**hdfsspout.hdfs** | | HDFS URI. Example:
hdfs://namenodehost:8020
-|**hdfsspout.source.dir** | | HDFS location from where to
read. E.g. /data/inputfiles |
-|**hdfsspout.archive.dir** | | After a file is processed
completely it will be moved to this directory. E.g. /data/done|
-|**hdfsspout.badfiles.dir** | | if there is an error parsing a
file's contents, the file is moved to this location. E.g. /data/badfiles |
-|hdfsspout.lock.dir | '.lock' subdirectory under
hdfsspout.source.dir | Dir in which lock files will be created. Concurrent HDFS
spout instances synchronize using *lock* files. Before processing a file the
spout instance creates a lock file in this directory with same name as input
file and deletes this lock file after processing the file. Spout also
periodically makes a note of its progress (wrt reading the input file) in the
lock file so that another spout instance can resume progress on the same file
if the spout dies for any reason. When a toplogy is killed, if a .lock/DIRLOCK
file is left behind it can be safely deleted to allow normal resumption of the
topology on restart.|
-|hdfsspout.ignore.suffix | .ignore | File names with this suffix in
the in the hdfsspout.source.dir location will not be processed|
-|hdfsspout.commit.count | 20000 | Record progress in the lock
file after these many records are processed. If set to 0, this criterion will
not be used. |
-|hdfsspout.commit.sec | 10 | Record progress in the lock
file after these many seconds have elapsed. Must be greater than 0 |
-|hdfsspout.max.outstanding | 10000 | Limits the number of unACKed
tuples by pausing tuple generation (if ACKers are used in the topology) |
-|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 |
-|hdfsspout.clocks.insync | true | Indicates whether clocks on the
storm machines are in sync (using services like NTP) |
-|hdfs.config (unless changed) | | Set it to a Map of Key/value
pairs indicating the HDFS settigns 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.|
-
+---
# HDFS Bolt
## Usage
@@ -189,7 +90,7 @@ By default, storm-hdfs uses the following Hadoop
dependencies:
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
- <version>2.2.0</version>
+ <version>2.6.1</version>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -200,7 +101,7 @@ By default, storm-hdfs uses the following Hadoop
dependencies:
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
- <version>2.2.0</version>
+ <version>2.6.1</version>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
@@ -509,7 +410,123 @@ On worker hosts the bolt/trident-state code will use the
keytab file with princi
Namenode. This method is little dangerous as you need to ensure all workers
have the keytab file at the same location and you need
to remember this as you bring up new hosts in the cluster.
-## License
+---
+
+# HDFS Spout
+
+Hdfs spout is intended to allow feeding data into Storm from a HDFS directory.
+It will actively monitor the directory to consume any new files that appear in
the directory.
+HDFS spout does not support Trident currently.
+
+**Impt**: Hdfs spout assumes that the files being made visible to it in the
monitored directory
+are NOT actively being written to. Only after a file is completely written
should it be made
+visible to the spout. This can be achieved by either writing the files out to
another directory
+and once completely written, move it to the monitored directory. Alternatively
the file
+can be created with a '.ignore' suffix in the monitored directory and after
data is completely
+written, rename it without the suffix. File names with a '.ignore' suffix are
ignored
+by the spout.
+
+When the spout is actively consuming a file, it renames the file with a
'.inprogress' suffix.
+After consuming all the contents in the file, the file will be moved to a
configurable *done*
+directory and the '.inprogress' suffix will be dropped.
+
+**Concurrency** If multiple spout instances are used in the topology, each
instance will consume
+a different file. Synchronization among spout instances is done using lock
files created in a
+(by default) '.lock' subdirectory under the monitored directory. A file with
the same name
+as the file being consumed (without the in progress suffix) is created in the
lock directory.
+Once the file is completely consumed, the corresponding lock file is deleted.
+
+**Recovery from failure**
+Periodically, the spout also records progress information wrt to how much of
the file has been
+consumed in the lock file. In case of an crash of the spout instance (or force
kill of topology)
+another spout can take over the file and resume from the location recorded in
the lock file.
+
+Certain error conditions (such spout crashing) can leave behind lock files
without deleting them.
+Such a stale lock file also indicates that the corresponding input file has
also not been completely
+processed. When detected, ownership of such stale lock files will be
transferred to another spout.
+The configuration 'hdfsspout.lock.timeout.sec' is used to specify the duration
of inactivity after
+which lock files should be considered stale. For lock file ownership transfer
to succeed, the HDFS
+lease on the file (from prev lock owner) should have expired. Spouts scan for
stale lock files
+before selecting the next file for consumption.
+
+**Lock on *.lock* Directory**
+Hdfs spout instances create a *DIRLOCK* file in the .lock directory to
co-ordinate certain accesses to
+the .lock dir itself. A spout will try to create it when it needs access to
the .lock directory and
+then delete it when done. In case of a topology crash or force kill, this
file may not get deleted.
+In this case it should be deleted manually to allow the new topology instance
to regain full access
+to the .lock directory and resume normal processing.
+
+## Usage
+
+The following example creates an HDFS spout that reads text files from HDFS
path hdfs://localhost:54310/source.
+
+```java
+// Instantiate spout
+HdfsSpout textReaderSpout = new
HdfsSpout().withOutputFields(TextFileReader.defaultFields);
+// HdfsSpout seqFileReaderSpout = new
HdfsSpout().withOutputFields(SequenceFileReader.defaultFields);
+
+// textReaderSpout.withConfigKey("custom.keyname"); // Optional. Not required
normally unless you need to change the keyname use to provide hds settings.
This keyname defaults to 'hdfs.config'
+
+// Configure it
+Config conf = new Config();
+conf.put(Configs.SOURCE_DIR, "hdfs://localhost:54310/source");
+conf.put(Configs.ARCHIVE_DIR, "hdfs://localhost:54310/done");
+conf.put(Configs.BAD_DIR, "hdfs://localhost:54310/badfiles");
+conf.put(Configs.READER_TYPE, "text"); // or 'seq' for sequence files
+
+// Create & configure topology
+TopologyBuilder builder = new TopologyBuilder();
+builder.setSpout("hdfsspout", textReaderSpout, SPOUT_NUM);
+
+// Setup bolts and other topology configuration
+ ..snip..
+
+// Submit topology with config
+StormSubmitter.submitTopologyWithProgressBar("topologyName", conf,
builder.createTopology());
+```
+
+See sample HdfsSpoutTopolgy in storm-starter.
+
+## Configuration Settings
+Class HdfsSpout provided following methods for configuration:
+
+`HdfsSpout withOutputFields(String... fields)` : This sets the names for the
output fields.
+The number of fields depends upon the reader being used. For convenience,
built-in reader types
+expose a static member called `defaultFields` that can be used for this.
+
+ `HdfsSpout withConfigKey(String configKey)`
+Optional setting. It allows overriding the default key name ('hdfs.config')
with new name for
+specifying HDFS configs. Typically used to specify kerberos keytab and
principal.
+
+**E.g:**
+```java
+ HashMap map = new HashMap();
+ map.put("hdfs.keytab.file", "/path/to/keytab");
+ map.put("hdfs.kerberos.principal","[email protected]");
+ conf.set("hdfs.config", map)
+```
+
+Only settings mentioned in **bold** are required.
+
+| Setting | Default | Description |
+|------------------------------|-------------|-------------|
+|**hdfsspout.reader.type** | | Indicates the reader for the
file format. Set to 'seq' for reading sequence files or 'text' for text files.
Set to a fully qualified class name if using a custom type (that implements
interface org.apache.storm.hdfs.spout.FileReader)|
+|**hdfsspout.hdfs** | | HDFS URI. Example:
hdfs://namenodehost:8020
+|**hdfsspout.source.dir** | | HDFS location from where to
read. E.g. /data/inputfiles |
+|**hdfsspout.archive.dir** | | After a file is processed
completely it will be moved to this directory. E.g. /data/done|
+|**hdfsspout.badfiles.dir** | | if there is an error parsing a
file's contents, the file is moved to this location. E.g. /data/badfiles |
+|hdfsspout.lock.dir | '.lock' subdirectory under
hdfsspout.source.dir | Dir in which lock files will be created. Concurrent HDFS
spout instances synchronize using *lock* files. Before processing a file the
spout instance creates a lock file in this directory with same name as input
file and deletes this lock file after processing the file. Spout also
periodically makes a note of its progress (wrt reading the input file) in the
lock file so that another spout instance can resume progress on the same file
if the spout dies for any reason. When a toplogy is killed, if a .lock/DIRLOCK
file is left behind it can be safely deleted to allow normal resumption of the
topology on restart.|
+|hdfsspout.ignore.suffix | .ignore | File names with this suffix in
the in the hdfsspout.source.dir location will not be processed|
+|hdfsspout.commit.count | 20000 | Record progress in the lock
file after these many records are processed. If set to 0, this criterion will
not be used. |
+|hdfsspout.commit.sec | 10 | Record progress in the lock
file after these many seconds have elapsed. Must be greater than 0 |
+|hdfsspout.max.outstanding | 10000 | Limits the number of unACKed
tuples by pausing tuple generation (if ACKers are used in the topology) |
+|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 |
+|hdfsspout.clocks.insync | true | Indicates whether clocks on the
storm machines are in sync (using services like NTP) |
+|hdfs.config (unless changed) | | Set it to a Map of Key/value
pairs indicating the HDFS settigns 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.|
+
+---
+
+# License
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
@@ -528,7 +545,7 @@ KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-## Committer Sponsors
+# Committer Sponsors
* P. Taylor Goetz ([[email protected]](mailto:[email protected]))
- * Bobby Evans ([[email protected]](mailto:[email protected]))
+ * Bobby Evans ([[email protected]](mailto:[email protected]))
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/AbstractFileReader.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/AbstractFileReader.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/AbstractFileReader.java
index 9996c6c..e1339df 100644
---
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/AbstractFileReader.java
+++
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/AbstractFileReader.java
@@ -18,7 +18,6 @@
package org.apache.storm.hdfs.spout;
-import backtype.storm.tuple.Fields;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -26,11 +25,14 @@ import org.apache.hadoop.fs.Path;
abstract class AbstractFileReader implements FileReader {
private final Path file;
- private Fields fields;
public AbstractFileReader(FileSystem fs, Path file) {
- if (fs == null || file == null)
- throw new IllegalArgumentException("file and filesystem args cannot be
null");
+ if (fs == null ) {
+ throw new IllegalArgumentException("filesystem arg cannot be null for
reader");
+ }
+ if (file == null ) {
+ throw new IllegalArgumentException("file arg cannot be null for reader");
+ }
this.file = file;
}
@@ -42,8 +44,8 @@ abstract class AbstractFileReader implements FileReader {
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
+ if (this == o) { return true; }
+ if (o == null || getClass() != o.getClass()) { return false; }
AbstractFileReader that = (AbstractFileReader) o;
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/DirLock.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/DirLock.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/DirLock.java
index cb8e015..25a136c 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/DirLock.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/DirLock.java
@@ -36,10 +36,11 @@ public class DirLock {
private FileSystem fs;
private final Path lockFile;
public static final String DIR_LOCK_FILE = "DIRLOCK";
- private static final Logger log = LoggerFactory.getLogger(DirLock.class);
+ private static final Logger LOG = LoggerFactory.getLogger(DirLock.class);
private DirLock(FileSystem fs, Path lockFile) throws IOException {
- if( fs.isDirectory(lockFile) )
+ if( fs.isDirectory(lockFile) ) {
throw new IllegalArgumentException(lockFile.toString() + " is not a
directory");
+ }
this.fs = fs;
this.lockFile = lockFile;
}
@@ -57,15 +58,15 @@ public class DirLock {
try {
FSDataOutputStream ostream = HdfsUtils.tryCreateFile(fs, lockFile);
if (ostream!=null) {
- log.info("Thread ({}) acquired lock on dir {}", threadInfo(), dir);
+ LOG.debug("Thread ({}) Acquired lock on dir {}", threadInfo(), dir);
ostream.close();
return new DirLock(fs, lockFile);
} else {
- log.info("Thread ({}) cannot lock dir {} as its already locked.",
threadInfo(), dir);
+ LOG.debug("Thread ({}) cannot lock dir {} as its already locked.",
threadInfo(), dir);
return null;
}
} catch (IOException e) {
- log.error("Error when acquiring lock on dir " + dir, e);
+ LOG.error("Error when acquiring lock on dir " + dir, e);
throw e;
}
}
@@ -82,10 +83,10 @@ public class DirLock {
/** Release lock on dir by deleting the lock file */
public void release() throws IOException {
if(!fs.delete(lockFile, false)) {
- log.error("Thread {} could not delete dir lock {} ", threadInfo(),
lockFile);
+ LOG.error("Thread {} could not delete dir lock {} ", threadInfo(),
lockFile);
}
else {
- log.info("Thread {} released dir lock {} ", threadInfo(), lockFile);
+ LOG.debug("Thread {} Released dir lock {} ", threadInfo(), lockFile);
}
}
@@ -98,8 +99,9 @@ public class DirLock {
try {
long modTime = fs.getFileStatus(dirLockFile).getModificationTime();
- if(modTime <= expiryTime)
+ if(modTime <= expiryTime) {
return takeOwnership(fs, dirLockFile);
+ }
return null;
} catch (IOException e) {
return null;
@@ -109,7 +111,7 @@ public class DirLock {
private static DirLock takeOwnership(FileSystem fs, Path dirLockFile) throws
IOException {
if(fs instanceof DistributedFileSystem) {
if (!((DistributedFileSystem) fs).recoverLease(dirLockFile)) {
- log.warn("Unable to recover lease on dir lock file " + dirLockFile + "
right now. Cannot transfer ownership. Will need to try later.");
+ LOG.warn("Unable to recover lease on dir lock file " + dirLockFile + "
right now. Cannot transfer ownership. Will need to try later.");
return null;
}
}
@@ -117,8 +119,9 @@ public class DirLock {
// delete and recreate lock file
if( fs.delete(dirLockFile, false) ) { // returns false if somebody else
already deleted it (to take ownership)
FSDataOutputStream ostream = HdfsUtils.tryCreateFile(fs, dirLockFile);
- if(ostream!=null)
+ if(ostream!=null) {
ostream.close();
+ }
return new DirLock(fs, dirLockFile);
}
return null;
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileLock.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileLock.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileLock.java
index c64336d..0217cf9 100644
---
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileLock.java
+++
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileLock.java
@@ -49,7 +49,7 @@ public class FileLock {
private final FSDataOutputStream lockFileStream;
private LogEntry lastEntry;
- private static final Logger log = LoggerFactory.getLogger(FileLock.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FileLock.class);
private FileLock(FileSystem fs, Path lockFile, FSDataOutputStream
lockFileStream, String spoutId)
throws IOException {
@@ -66,7 +66,7 @@ public class FileLock {
this.lockFile = lockFile;
this.lockFileStream = fs.append(lockFile);
this.componentID = spoutId;
- log.debug("Acquired abandoned lockFile {}, Spout {}", lockFile, spoutId);
+ LOG.info("Acquired abandoned lockFile {}, Spout {}", lockFile, spoutId);
logProgress(entry.fileOffset, true);
}
@@ -81,10 +81,12 @@ public class FileLock {
long now = System.currentTimeMillis();
LogEntry entry = new LogEntry(now, componentID, fileOffset);
String line = entry.toString();
- if(prefixNewLine)
+ if(prefixNewLine) {
lockFileStream.writeBytes(System.lineSeparator() + line);
- else
+ }
+ else {
lockFileStream.writeBytes(line);
+ }
lockFileStream.hflush();
lastEntry = entry; // update this only after writing to hdfs
@@ -95,11 +97,11 @@ public class FileLock {
*/
public void release() throws IOException {
lockFileStream.close();
- if(!fs.delete(lockFile, false)){
- log.warn("Unable to delete lock file, Spout = {}", componentID);
+ if(!fs.delete(lockFile, false)) {
+ LOG.warn("Unable to delete lock file, Spout = {}", componentID);
throw new IOException("Unable to delete lock file");
}
- log.debug("Released lock file {}. Spout {}", lockFile, componentID);
+ LOG.debug("Released lock file {}. Spout {}", lockFile, componentID);
}
// For testing only.. invoked via reflection
@@ -116,14 +118,14 @@ public class FileLock {
try {
FSDataOutputStream ostream = HdfsUtils.tryCreateFile(fs, lockFile);
if (ostream != null) {
- log.debug("Acquired lock on file {}. LockFile= {}, Spout = {}",
fileToLock, lockFile, spoutId);
+ LOG.debug("Acquired lock on file {}. LockFile= {}, Spout = {}",
fileToLock, lockFile, spoutId);
return new FileLock(fs, lockFile, ostream, spoutId);
} else {
- log.debug("Cannot lock file {} as its already locked. Spout = {}",
fileToLock, spoutId);
+ LOG.debug("Cannot lock file {} as its already locked. Spout = {}",
fileToLock, spoutId);
return null;
}
} catch (IOException e) {
- log.error("Error when acquiring lock on file " + fileToLock + " Spout =
" + spoutId, e);
+ LOG.error("Error when acquiring lock on file " + fileToLock + " Spout =
" + spoutId, e);
throw e;
}
}
@@ -188,7 +190,7 @@ public class FileLock {
try {
if(fs instanceof DistributedFileSystem ) {
if( !((DistributedFileSystem) fs).recoverLease(lockFile) ) {
- log.warn("Unable to recover lease on lock file {} right now. Cannot
transfer ownership. Will need to try later. Spout = {}" , lockFile , spoutId);
+ LOG.warn("Unable to recover lease on lock file {} right now. Cannot
transfer ownership. Will need to try later. Spout = {}", lockFile, spoutId);
return null;
}
}
@@ -196,10 +198,10 @@ public class FileLock {
} catch (IOException e) {
if (e instanceof RemoteException &&
((RemoteException) e).unwrapRemoteException() instanceof
AlreadyBeingCreatedException) {
- log.warn("Lock file " + lockFile + "is currently open. Cannot
transfer ownership now. Will need to try later. Spout= " + spoutId, e);
+ LOG.warn("Lock file " + lockFile + "is currently open. Cannot transfer
ownership now. Will need to try later. Spout= " + spoutId, e);
return null;
} else { // unexpected error
- log.warn("Cannot transfer ownership now for lock file " + lockFile +
". Will need to try later. Spout =" + spoutId, e);
+ LOG.warn("Cannot transfer ownership now for lock file " + lockFile +
". Will need to try later. Spout =" + spoutId, e);
throw e;
}
}
@@ -224,17 +226,20 @@ public class FileLock {
// locate expired lock files (if any). Try to take ownership (oldest lock
first)
for (Path file : listing) {
- if(file.getName().equalsIgnoreCase( DirLock.DIR_LOCK_FILE) )
+ if(file.getName().equalsIgnoreCase( DirLock.DIR_LOCK_FILE) ) {
continue;
+ }
LogEntry lastEntry = getLastEntryIfStale(fs, file, olderThan);
if(lastEntry!=null) {
FileLock lock = FileLock.takeOwnership(fs, file, lastEntry, spoutId);
- if(lock!=null)
+ if(lock!=null) {
return lock;
+ }
}
}
- if(listing.isEmpty())
- log.info("No abandoned lock files found by Spout {}", spoutId);
+ if(listing.isEmpty()) {
+ LOG.debug("No abandoned lock files found by Spout {}", spoutId);
+ }
return null;
}
@@ -259,13 +264,15 @@ public class FileLock {
// locate oldest expired lock file (if any) and take ownership
for (Path file : listing) {
- if(file.getName().equalsIgnoreCase( DirLock.DIR_LOCK_FILE) )
+ if(file.getName().equalsIgnoreCase( DirLock.DIR_LOCK_FILE) ) {
continue;
+ }
LogEntry lastEntry = getLastEntryIfStale(fs, file, olderThan);
- if(lastEntry!=null)
+ if(lastEntry!=null) {
return new HdfsUtils.Pair<>(file, lastEntry);
+ }
}
- log.info("No abandoned files found");
+ LOG.debug("No abandoned files found");
return null;
}
@@ -299,13 +306,13 @@ public class FileLock {
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof LogEntry)) return false;
+ if (this == o) { return true; }
+ if (!(o instanceof LogEntry)) { return false; }
LogEntry logEntry = (LogEntry) o;
- if (eventTime != logEntry.eventTime) return false;
- if (!componentID.equals(logEntry.componentID)) return false;
+ if (eventTime != logEntry.eventTime) { return false; }
+ if (!componentID.equals(logEntry.componentID)) { return false; }
return fileOffset.equals(logEntry.fileOffset);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/HdfsSpout.java
----------------------------------------------------------------------
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 65a49f3..93d08d5 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
@@ -66,7 +66,7 @@ public class HdfsSpout extends BaseRichSpout {
private String ignoreSuffix = ".ignore";
// other members
- private static final Logger log = LoggerFactory.getLogger(HdfsSpout.class);
+ private static final Logger LOG = LoggerFactory.getLogger(HdfsSpout.class);
private ProgressTracker tracker = null;
@@ -119,17 +119,17 @@ public class HdfsSpout extends BaseRichSpout {
}
public void nextTuple() {
- log.debug("Next Tuple {}", spoutId);
+ LOG.trace("Next Tuple {}", spoutId);
// 1) First re-emit any previously failed tuples (from retryList)
if (!retryList.isEmpty()) {
- log.debug("Sending from retry list");
+ LOG.debug("Sending tuple from retry list");
HdfsUtils.Pair<MessageId, List<Object>> pair = retryList.remove();
emitData(pair.getValue(), pair.getKey());
return;
}
if( ackEnabled && tracker.size()>= maxOutstanding) {
- log.warn("Waiting for more ACKs before generating new tuples. " +
+ LOG.warn("Waiting for more ACKs before generating new tuples. " +
"Progress tracker size has reached limit {}, SpoutID {}"
, maxOutstanding, spoutId);
// Don't emit anything .. allow configured spout wait strategy to kick in
@@ -143,7 +143,7 @@ public class HdfsSpout extends BaseRichSpout {
if (reader == null) {
reader = pickNextFile();
if (reader == null) {
- log.debug("Currently no new files to process under : " +
sourceDirPath);
+ LOG.debug("Currently no new files to process under : " +
sourceDirPath);
return;
} else {
fileReadCompletely=false;
@@ -174,11 +174,11 @@ public class HdfsSpout extends BaseRichSpout {
}
}
} catch (IOException e) {
- log.error("I/O Error processing at file location " +
getFileProgress(reader), e);
+ LOG.error("I/O Error processing at file location " +
getFileProgress(reader), e);
// don't emit anything .. allow configured spout wait strategy to kick
in
return;
} catch (ParseException e) {
- log.error("Parsing error when processing at file location " +
getFileProgress(reader) +
+ LOG.error("Parsing error when processing at file location " +
getFileProgress(reader) +
". Skipping remainder of file.", e);
markFileAsBad(reader.getFilePath());
// Note: We don't return from this method on ParseException to avoid
triggering the
@@ -190,23 +190,27 @@ public class HdfsSpout extends BaseRichSpout {
// will commit progress into lock file if commit threshold is reached
private void commitProgress(FileOffset position) {
- if(position==null)
+ if(position==null) {
return;
+ }
if ( lock!=null && canCommitNow() ) {
try {
- lock.heartbeat(position.toString());
+ String pos = position.toString();
+ lock.heartbeat(pos);
+ LOG.debug("{} Committed progress. {}", spoutId, pos);
acksSinceLastCommit = 0;
commitTimeElapsed.set(false);
setupCommitElapseTimer();
} catch (IOException e) {
- log.error("Unable to commit progress Will retry later. Spout ID = " +
spoutId, e);
+ LOG.error("Unable to commit progress Will retry later. Spout ID = " +
spoutId, e);
}
}
}
private void setupCommitElapseTimer() {
- if(commitFrequencySec<=0)
+ if(commitFrequencySec<=0) {
return;
+ }
TimerTask timerTask = new TimerTask() {
@Override
public void run() {
@@ -223,9 +227,9 @@ public class HdfsSpout extends BaseRichSpout {
private void markFileAsDone(Path filePath) {
try {
Path newFile = renameCompletedFile(reader.getFilePath());
- log.info("Completed processing {}. Spout Id = {} ", newFile, spoutId);
+ LOG.info("Completed processing {}. Spout Id = {}", newFile, spoutId);
} catch (IOException e) {
- log.error("Unable to archive completed file" + filePath + " Spout ID " +
spoutId, e);
+ LOG.error("Unable to archive completed file" + filePath + " Spout ID " +
spoutId, e);
}
closeReaderAndResetTrackers();
}
@@ -236,13 +240,13 @@ public class HdfsSpout extends BaseRichSpout {
String originalName = new Path(fileNameMinusSuffix).getName();
Path newFile = new Path( badFilesDirPath + Path.SEPARATOR + originalName);
- log.info("Moving bad file {} to {}. Processed it till offset {}. SpoutID=
{}", originalName, newFile, tracker.getCommitPosition(), spoutId);
+ LOG.info("Moving bad file {} to {}. Processed it till offset {}. SpoutID=
{}", originalName, newFile, tracker.getCommitPosition(), spoutId);
try {
if (!hdfs.rename(file, newFile) ) { // seems this can fail by returning
false or throwing exception
throw new IOException("Move failed for bad file: " + file); // convert
false ret value to exception
}
} catch (IOException e) {
- log.warn("Error moving bad file: " + file + " to destination " + newFile
+ " SpoutId =" + spoutId, e);
+ LOG.warn("Error moving bad file: " + file + " to destination " + newFile
+ " SpoutId =" + spoutId, e);
}
closeReaderAndResetTrackers();
}
@@ -256,21 +260,21 @@ public class HdfsSpout extends BaseRichSpout {
reader = null;
try {
lock.release();
- log.debug("Spout {} released FileLock. SpoutId = {}",
lock.getLockFile(), spoutId);
+ LOG.debug("Spout {} released FileLock. SpoutId = {}",
lock.getLockFile(), spoutId);
} catch (IOException e) {
- log.error("Unable to delete lock file : " + this.lock.getLockFile() + "
SpoutId =" + spoutId, e);
+ LOG.error("Unable to delete lock file : " + this.lock.getLockFile() + "
SpoutId =" + spoutId, e);
}
lock = null;
}
protected void emitData(List<Object> tuple, MessageId id) {
- log.debug("Emitting - {}", id);
+ LOG.trace("Emitting - {}", id);
this.collector.emit(tuple, id);
inflight.put(id, tuple);
}
public void open(Map conf, TopologyContext context, SpoutOutputCollector
collector) {
- log.info("Opening HDFS Spout {}", spoutId);
+ LOG.info("Opening HDFS Spout");
this.conf = conf;
this.commitTimer = new Timer();
this.tracker = new ProgressTracker();
@@ -290,7 +294,7 @@ public class HdfsSpout extends BaseRichSpout {
try {
this.hdfs = FileSystem.get(URI.create(hdfsUri), hdfsConfig);
} catch (IOException e) {
- log.error("Unable to instantiate file system", e);
+ LOG.error("Unable to instantiate file system", e);
throw new RuntimeException("Unable to instantiate file system", e);
}
@@ -299,13 +303,13 @@ public class HdfsSpout extends BaseRichSpout {
Map<String, Object> map = (Map<String, Object>)conf.get(configKey);
if(map != null) {
for(String keyName : map.keySet()){
- log.info("HDFS Config override : " + keyName + " = " +
String.valueOf(map.get(keyName)));
+ LOG.info("HDFS Config override : " + keyName + " = " +
String.valueOf(map.get(keyName)));
this.hdfsConfig.set(keyName, String.valueOf(map.get(keyName)));
}
try {
HdfsSecurityUtil.login(conf, hdfsConfig);
} catch (IOException e) {
- log.error("HDFS Login failed ", e);
+ LOG.error("HDFS Login failed ", e);
throw new RuntimeException(e);
}
} // if(map != null)
@@ -319,14 +323,14 @@ public class HdfsSpout extends BaseRichSpout {
// -- source dir config
if ( !conf.containsKey(Configs.SOURCE_DIR) ) {
- log.error(Configs.SOURCE_DIR + " setting is required");
+ LOG.error(Configs.SOURCE_DIR + " setting is required");
throw new RuntimeException(Configs.SOURCE_DIR + " setting is required");
}
this.sourceDirPath = new Path( conf.get(Configs.SOURCE_DIR).toString() );
// -- archive dir config
if ( !conf.containsKey(Configs.ARCHIVE_DIR) ) {
- log.error(Configs.ARCHIVE_DIR + " setting is required");
+ LOG.error(Configs.ARCHIVE_DIR + " setting is required");
throw new RuntimeException(Configs.ARCHIVE_DIR + " setting is required");
}
this.archiveDirPath = new Path( conf.get(Configs.ARCHIVE_DIR).toString() );
@@ -334,7 +338,7 @@ public class HdfsSpout extends BaseRichSpout {
// -- bad files dir config
if ( !conf.containsKey(Configs.BAD_DIR) ) {
- log.error(Configs.BAD_DIR + " setting is required");
+ LOG.error(Configs.BAD_DIR + " setting is required");
throw new RuntimeException(Configs.BAD_DIR + " setting is required");
}
@@ -352,27 +356,35 @@ public class HdfsSpout extends BaseRichSpout {
validateOrMakeDir(hdfs,lockDirPath,"locks");
// -- lock timeout
- if( conf.get(Configs.LOCK_TIMEOUT) !=null )
- this.lockTimeoutSec =
Integer.parseInt(conf.get(Configs.LOCK_TIMEOUT).toString());
+ if( conf.get(Configs.LOCK_TIMEOUT) !=null ) {
+ this.lockTimeoutSec =
Integer.parseInt(conf.get(Configs.LOCK_TIMEOUT).toString());
+ }
// -- enable/disable ACKing
Object ackers = conf.get(Config.TOPOLOGY_ACKER_EXECUTORS);
- if( ackers!=null )
- this.ackEnabled = ( Integer.parseInt( ackers.toString() ) > 0 );
- else
+ if( ackers!=null ) {
+ int ackerCount = Integer.parseInt(ackers.toString());
+ this.ackEnabled = (ackerCount>0);
+ LOG.debug("ACKer count = {}", ackerCount);
+ }
+ else {
this.ackEnabled = false;
+ LOG.debug("No ACKers config found");
+ }
- log.info("ACK mode is {}", ackEnabled ? "enabled" : "disabled");
+ LOG.info("ACK mode is {}", ackEnabled ? "enabled" : "disabled");
// -- commit frequency - count
- if( conf.get(Configs.COMMIT_FREQ_COUNT) != null )
- commitFrequencyCount = Integer.parseInt(
conf.get(Configs.COMMIT_FREQ_COUNT).toString() );
+ if( conf.get(Configs.COMMIT_FREQ_COUNT) != null ) {
+ commitFrequencyCount =
Integer.parseInt(conf.get(Configs.COMMIT_FREQ_COUNT).toString());
+ }
// -- commit frequency - seconds
if( conf.get(Configs.COMMIT_FREQ_SEC) != null ) {
commitFrequencySec =
Integer.parseInt(conf.get(Configs.COMMIT_FREQ_SEC).toString());
- if(commitFrequencySec<=0)
+ if(commitFrequencySec<=0) {
throw new RuntimeException(Configs.COMMIT_FREQ_SEC + " setting must be
greater than 0");
+ }
}
// -- max duplicate
@@ -394,15 +406,15 @@ public class HdfsSpout extends BaseRichSpout {
try {
if(fs.exists(dir)) {
if(! fs.isDirectory(dir) ) {
- log.error(dirDescription + " directory is a file, not a dir. " +
dir);
+ LOG.error(dirDescription + " directory is a file, not a dir. " +
dir);
throw new RuntimeException(dirDescription + " directory is a file,
not a dir. " + dir);
}
} else if(! fs.mkdirs(dir) ) {
- log.error("Unable to create " + dirDescription + " directory " + dir);
+ LOG.error("Unable to create " + dirDescription + " directory " + dir);
throw new RuntimeException("Unable to create " + dirDescription + "
directory " + dir);
}
} catch (IOException e) {
- log.error("Unable to create " + dirDescription + " directory " + dir, e);
+ LOG.error("Unable to create " + dirDescription + " directory " + dir, e);
throw new RuntimeException("Unable to create " + dirDescription + "
directory " + dir, e);
}
}
@@ -419,21 +431,20 @@ public class HdfsSpout extends BaseRichSpout {
classType.getConstructor(FileSystem.class, Path.class, Map.class);
return;
} catch (ClassNotFoundException e) {
- log.error(readerType + " not found in classpath.", e);
+ LOG.error(readerType + " not found in classpath.", e);
throw new IllegalArgumentException(readerType + " not found in
classpath.", e);
} catch (NoSuchMethodException e) {
- log.error(readerType + " is missing the expected constructor for
Readers.", e);
+ LOG.error(readerType + " is missing the expected constructor for
Readers.", e);
throw new IllegalArgumentException(readerType + " is missing the
expected constuctor for Readers.");
}
}
@Override
public void ack(Object msgId) {
+ LOG.trace("Ack received for msg {} on spout {}", msgId, spoutId);
if(!ackEnabled) {
- log.debug("Ack() called but acker count = 0", msgId, spoutId);
return;
}
- log.debug("Ack received for msg {} on spout {}", msgId, spoutId);
MessageId id = (MessageId) msgId;
inflight.remove(id);
++acksSinceLastCommit;
@@ -448,17 +459,20 @@ public class HdfsSpout extends BaseRichSpout {
private boolean canCommitNow() {
- if( commitFrequencyCount>0 && acksSinceLastCommit >= commitFrequencyCount
)
+ if( commitFrequencyCount>0 && acksSinceLastCommit >= commitFrequencyCount
) {
return true;
+ }
return commitTimeElapsed.get();
}
@Override
public void fail(Object msgId) {
- log.debug("Fail() called for msg {} on spout {}", msgId, spoutId);
+ LOG.trace("Fail received for msg id {} on spout {}", msgId, spoutId);
super.fail(msgId);
- HdfsUtils.Pair<MessageId, List<Object>> item = HdfsUtils.Pair.of(msgId,
inflight.remove(msgId));
- retryList.add(item);
+ if(ackEnabled) {
+ HdfsUtils.Pair<MessageId, List<Object>> item = HdfsUtils.Pair.of(msgId,
inflight.remove(msgId));
+ retryList.add(item);
+ }
}
private FileReader pickNextFile() {
@@ -466,10 +480,10 @@ public class HdfsSpout extends BaseRichSpout {
// 1) If there are any abandoned files, pick oldest one
lock = getOldestExpiredLock();
if (lock != null) {
- log.debug("Spout {} now took over ownership of abandoned FileLock {}",
spoutId, lock.getLockFile());
+ LOG.debug("Spout {} now took over ownership of abandoned FileLock {}",
spoutId, lock.getLockFile());
Path file = getFileForLockFile(lock.getLockFile(), sourceDirPath);
String resumeFromOffset = lock.getLastLogEntry().fileOffset;
- log.info("Resuming processing of abandoned file : {}", file);
+ LOG.info("Resuming processing of abandoned file : {}", file);
return createFileReader(file, resumeFromOffset);
}
@@ -477,24 +491,25 @@ public class HdfsSpout extends BaseRichSpout {
Collection<Path> listing = HdfsUtils.listFilesByModificationTime(hdfs,
sourceDirPath, 0);
for (Path file : listing) {
- if( file.getName().endsWith(inprogress_suffix) )
+ if( file.getName().endsWith(inprogress_suffix) ) {
continue;
- if( file.getName().endsWith(ignoreSuffix) )
+ }
+ if( file.getName().endsWith(ignoreSuffix) ) {
continue;
-
+ }
lock = FileLock.tryLock(hdfs, file, lockDirPath, spoutId);
if( lock==null ) {
- log.debug("Unable to get FileLock, so skipping file: {}", file);
+ LOG.debug("Unable to get FileLock for {}, so skipping it.", file);
continue; // could not lock, so try another file.
}
- log.info("Processing : {} ", file);
+ LOG.info("Processing : {} ", file);
Path newFile = renameSelectedFile(file);
return createFileReader(newFile);
}
return null;
} catch (IOException e) {
- log.error("Unable to select next file for consumption " + sourceDirPath,
e);
+ LOG.error("Unable to select next file for consumption " + sourceDirPath,
e);
return null;
}
}
@@ -511,18 +526,19 @@ public class HdfsSpout extends BaseRichSpout {
if (dirlock == null) {
dirlock = DirLock.takeOwnershipIfStale(hdfs, lockDirPath,
lockTimeoutSec);
if (dirlock == null) {
- log.debug("Spout {} could not take over ownership of DirLock for {}",
spoutId, lockDirPath);
+ LOG.debug("Spout {} could not take over ownership of DirLock for {}",
spoutId, lockDirPath);
return null;
}
- log.debug("Spout {} now took over ownership of abandoned DirLock for
{}", spoutId, lockDirPath);
+ LOG.debug("Spout {} now took over ownership of abandoned DirLock for
{}", spoutId, lockDirPath);
} else {
- log.debug("Spout {} now owns DirLock for {}", spoutId, lockDirPath);
+ LOG.debug("Spout {} now owns DirLock for {}", spoutId, lockDirPath);
}
try {
// 2 - if clocks are in sync then simply take ownership of the oldest
expired lock
- if (clocksInSync)
+ if (clocksInSync) {
return FileLock.acquireOldestExpiredLock(hdfs, lockDirPath,
lockTimeoutSec, spoutId);
+ }
// 3 - if clocks are not in sync ..
if( lastExpiredLock == null ) {
@@ -532,8 +548,9 @@ public class HdfsSpout extends BaseRichSpout {
return null;
}
// see if lockTimeoutSec time has elapsed since we last selected the
lock file
- if( hasExpired(lastExpiredLockTime) )
+ if( hasExpired(lastExpiredLockTime) ) {
return null;
+ }
// If lock file has expired, then own it
FileLock.LogEntry lastEntry = FileLock.getLastEntry(hdfs,
lastExpiredLock.getKey());
@@ -548,7 +565,7 @@ public class HdfsSpout extends BaseRichSpout {
}
} finally {
dirlock.release();
- log.debug("Released DirLock {}, SpoutID {} ", dirlock.getLockFile(),
spoutId);
+ LOG.debug("Released DirLock {}, SpoutID {} ", dirlock.getLockFile(),
spoutId);
}
}
@@ -564,17 +581,18 @@ public class HdfsSpout extends BaseRichSpout {
*/
private FileReader createFileReader(Path file)
throws IOException {
- if(readerType.equalsIgnoreCase(Configs.SEQ))
+ if(readerType.equalsIgnoreCase(Configs.SEQ)) {
return new SequenceFileReader(this.hdfs, file, conf);
- if(readerType.equalsIgnoreCase(Configs.TEXT))
+ }
+ if(readerType.equalsIgnoreCase(Configs.TEXT)) {
return new TextFileReader(this.hdfs, file, conf);
-
+ }
try {
Class<?> clsType = Class.forName(readerType);
Constructor<?> constructor = clsType.getConstructor(FileSystem.class,
Path.class, Map.class);
return (FileReader) constructor.newInstance(this.hdfs, file, conf);
} catch (Exception e) {
- log.error(e.getMessage(), e);
+ LOG.error(e.getMessage(), e);
throw new RuntimeException("Unable to instantiate " + readerType, e);
}
}
@@ -589,17 +607,19 @@ public class HdfsSpout extends BaseRichSpout {
*/
private FileReader createFileReader(Path file, String offset)
throws IOException {
- if(readerType.equalsIgnoreCase(Configs.SEQ))
+ if(readerType.equalsIgnoreCase(Configs.SEQ)) {
return new SequenceFileReader(this.hdfs, file, conf, offset);
- if(readerType.equalsIgnoreCase(Configs.TEXT))
+ }
+ if(readerType.equalsIgnoreCase(Configs.TEXT)) {
return new TextFileReader(this.hdfs, file, conf, offset);
+ }
try {
Class<?> clsType = Class.forName(readerType);
Constructor<?> constructor = clsType.getConstructor(FileSystem.class,
Path.class, Map.class, String.class);
return (FileReader) constructor.newInstance(this.hdfs, file, conf,
offset);
} catch (Exception e) {
- log.error(e.getMessage(), e);
+ LOG.error(e.getMessage(), e);
throw new RuntimeException("Unable to instantiate " + readerType, e);
}
}
@@ -621,11 +641,13 @@ public class HdfsSpout extends BaseRichSpout {
throws IOException {
String lockFileName = lockFile.getName();
Path dataFile = new Path(sourceDirPath + Path.SEPARATOR + lockFileName +
inprogress_suffix);
- if( hdfs.exists(dataFile) )
+ if( hdfs.exists(dataFile) ) {
return dataFile;
+ }
dataFile = new Path(sourceDirPath + Path.SEPARATOR + lockFileName);
- if(hdfs.exists(dataFile))
+ if(hdfs.exists(dataFile)) {
return dataFile;
+ }
return null;
}
@@ -637,11 +659,11 @@ public class HdfsSpout extends BaseRichSpout {
String newName = new Path(fileNameMinusSuffix).getName();
Path newFile = new Path( archiveDirPath + Path.SEPARATOR + newName );
- log.info("Completed consuming file {}", fileNameMinusSuffix);
+ LOG.info("Completed consuming file {}", fileNameMinusSuffix);
if (!hdfs.rename(file, newFile) ) {
throw new IOException("Rename failed for file: " + file);
}
- log.debug("Renamed completed file {} to {} ", file, newFile);
+ LOG.debug("Renamed file {} to {} ", file, newFile);
return newFile;
}
@@ -667,10 +689,12 @@ public class HdfsSpout extends BaseRichSpout {
@Override
public int compareTo(MessageId rhs) {
- if (msgNumber<rhs.msgNumber)
+ if (msgNumber<rhs.msgNumber) {
return -1;
- if(msgNumber>rhs.msgNumber)
+ }
+ if(msgNumber>rhs.msgNumber) {
return 1;
+ }
return 0;
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/ProgressTracker.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/ProgressTracker.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/ProgressTracker.java
index 2079ef4..d7de3ed 100644
---
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/ProgressTracker.java
+++
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/ProgressTracker.java
@@ -26,8 +26,9 @@ public class ProgressTracker {
TreeSet<FileOffset> offsets = new TreeSet<>();
public void recordAckedOffset(FileOffset newOffset) {
- if(newOffset==null)
+ if(newOffset==null) {
return;
+ }
offsets.add(newOffset);
FileOffset currHead = offsets.first();
@@ -40,8 +41,9 @@ public class ProgressTracker {
// remove contiguous elements from the head of the heap
// e.g.: 1,2,3,4,10,11,12,15 => 4,10,11,12,15
private void trimHead() {
- if(offsets.size()<=1)
+ if(offsets.size()<=1) {
return;
+ }
FileOffset head = offsets.first();
FileOffset head2 = offsets.higher(head);
if( head.isNextOffset(head2) ) {
@@ -52,8 +54,9 @@ public class ProgressTracker {
}
public FileOffset getCommitPosition() {
- if(!offsets.isEmpty())
+ if(!offsets.isEmpty()) {
return offsets.first().clone();
+ }
return null;
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/SequenceFileReader.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/SequenceFileReader.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/SequenceFileReader.java
index 580993b..7ed8639 100644
---
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/SequenceFileReader.java
+++
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/SequenceFileReader.java
@@ -18,7 +18,6 @@
package org.apache.storm.hdfs.spout;
-import backtype.storm.tuple.Fields;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
@@ -35,7 +34,7 @@ import java.util.Map;
public class SequenceFileReader<Key extends Writable,Value extends Writable>
extends AbstractFileReader {
- private static final Logger log = LoggerFactory
+ private static final Logger LOG = LoggerFactory
.getLogger(SequenceFileReader.class);
public static final String[] defaultFields = {"key", "value"};
private static final int DEFAULT_BUFF_SIZE = 4096;
@@ -93,7 +92,7 @@ public class SequenceFileReader<Key extends Writable,Value
extends Writable>
try {
reader.close();
} catch (IOException e) {
- log.warn("Ignoring error when closing file " + getFilePath(), e);
+ LOG.warn("Ignoring error when closing file " + getFilePath(), e);
}
}
@@ -124,8 +123,9 @@ public class SequenceFileReader<Key extends Writable,Value
extends Writable>
public Offset(String offset) {
try {
- if(offset==null)
+ if(offset==null) {
throw new IllegalArgumentException("offset cannot be null");
+ }
if(offset.equalsIgnoreCase("0")) {
this.lastSyncPoint = 0;
this.recordsSinceLastSync = 0;
@@ -168,17 +168,19 @@ public class SequenceFileReader<Key extends
Writable,Value extends Writable>
@Override
public int compareTo(FileOffset o) {
Offset rhs = ((Offset) o);
- if(currentRecord<rhs.currentRecord)
+ if(currentRecord<rhs.currentRecord) {
return -1;
- if(currentRecord==rhs.currentRecord)
+ }
+ if(currentRecord==rhs.currentRecord) {
return 0;
+ }
return 1;
}
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof Offset)) return false;
+ if (this == o) { return true; }
+ if (!(o instanceof Offset)) { return false; }
Offset offset = (Offset) o;
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/TextFileReader.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/TextFileReader.java
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/TextFileReader.java
index 641ac74..ac72d69 100644
---
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/TextFileReader.java
+++
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/TextFileReader.java
@@ -18,7 +18,6 @@
package org.apache.storm.hdfs.spout;
-import backtype.storm.tuple.Fields;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -41,7 +40,7 @@ class TextFileReader extends AbstractFileReader {
private static final int DEFAULT_BUFF_SIZE = 4096;
private BufferedReader reader;
- private final Logger log = LoggerFactory.getLogger(TextFileReader.class);
+ private final Logger LOG = LoggerFactory.getLogger(TextFileReader.class);
private TextFileReader.Offset offset;
public TextFileReader(FileSystem fs, Path file, Map conf) throws IOException
{
@@ -61,8 +60,9 @@ class TextFileReader extends AbstractFileReader {
String charSet = (conf==null || !conf.containsKey(CHARSET) ) ? "UTF-8" :
conf.get(CHARSET).toString();
int buffSz = (conf==null || !conf.containsKey(BUFFER_SIZE) ) ?
DEFAULT_BUFF_SIZE : Integer.parseInt( conf.get(BUFFER_SIZE).toString() );
reader = new BufferedReader(new InputStreamReader(in, charSet), buffSz);
- if(offset.charOffset >0)
+ if(offset.charOffset >0) {
reader.skip(offset.charOffset);
+ }
}
@@ -91,8 +91,9 @@ class TextFileReader extends AbstractFileReader {
sb.append((char)ch);
}
}
- if(before==offset.charOffset) // reached EOF, didnt read anything
+ if(before==offset.charOffset) { // reached EOF, didnt read anything
return null;
+ }
return sb.toString();
}
@@ -101,7 +102,7 @@ class TextFileReader extends AbstractFileReader {
try {
reader.close();
} catch (IOException e) {
- log.warn("Ignoring error when closing file " + getFilePath(), e);
+ LOG.warn("Ignoring error when closing file " + getFilePath(), e);
}
}
@@ -115,8 +116,9 @@ class TextFileReader extends AbstractFileReader {
}
public Offset(String offset) {
- if(offset==null)
+ if(offset==null) {
throw new IllegalArgumentException("offset cannot be null");
+ }
try {
if(offset.equalsIgnoreCase("0")) {
this.charOffset = 0;
@@ -154,17 +156,19 @@ class TextFileReader extends AbstractFileReader {
@Override
public int compareTo(FileOffset o) {
Offset rhs = ((Offset)o);
- if(lineNumber < rhs.lineNumber)
+ if(lineNumber < rhs.lineNumber) {
return -1;
- if(lineNumber == rhs.lineNumber)
+ }
+ if(lineNumber == rhs.lineNumber) {
return 0;
+ }
return 1;
}
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof Offset)) return false;
+ if (this == o) { return true; }
+ if (!(o instanceof Offset)) { return false; }
Offset that = (Offset) o;
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestDirLock.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestDirLock.java
b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestDirLock.java
index a7b73d6..b96f1ff 100644
---
a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestDirLock.java
+++
b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestDirLock.java
@@ -98,8 +98,9 @@ public class TestDirLock {
DirLockingThread[] thds = startThreads(100, locksDir);
for (DirLockingThread thd : thds) {
thd.join();
- if( !thd.cleanExit)
+ if( !thd.cleanExit ) {
System.err.println(thd.getName() + " did not exit cleanly");
+ }
Assert.assertTrue(thd.cleanExit);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestFileLock.java
----------------------------------------------------------------------
diff --git
a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestFileLock.java
b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestFileLock.java
index 7995248..725fa11 100644
---
a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestFileLock.java
+++
b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestFileLock.java
@@ -165,8 +165,9 @@ public class TestFileLock {
FileLockingThread[] thds = startThreads(100, file1, locksDir);
for (FileLockingThread thd : thds) {
thd.join();
- if( !thd.cleanExit)
+ if( !thd.cleanExit) {
System.err.println(thd.getName() + " did not exit cleanly");
+ }
Assert.assertTrue(thd.cleanExit);
}
@@ -325,8 +326,9 @@ public class TestFileLock {
FSDataInputStream os = null;
try {
os = fs.open(file);
- if (os == null)
+ if (os == null) {
return null;
+ }
BufferedReader reader = new BufferedReader(new InputStreamReader(os));
ArrayList<String> lines = new ArrayList<>();
for (String line = reader.readLine(); line != null; line =
reader.readLine()) {
@@ -336,8 +338,9 @@ public class TestFileLock {
} catch( FileNotFoundException e) {
return null;
} finally {
- if(os!=null)
+ if(os!=null) {
os.close();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/spout/TestHdfsSpout.java
----------------------------------------------------------------------
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 cdd4020..0412126 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
@@ -190,7 +190,7 @@ public class TestHdfsSpout {
// check lock file contents
List<String> contents = readTextFile(fs, lock.getLockFile().toString());
- System.err.println(contents);
+ Assert.assertFalse(contents.isEmpty());
// finish up reading the file
res2 = runSpout(spout2, "r2");
@@ -237,7 +237,7 @@ public class TestHdfsSpout {
// check lock file contents
List<String> contents = getTextFileContents(fs, lock.getLockFile());
- System.err.println(contents);
+ Assert.assertFalse(contents.isEmpty());
// finish up reading the file
res2 = runSpout(spout2, "r3");
@@ -309,11 +309,9 @@ public class TestHdfsSpout {
private List<String> listDir(Path p) throws IOException {
ArrayList<String> result = new ArrayList<>();
- System.err.println("*** Listing " + p);
RemoteIterator<LocatedFileStatus> fileNames = fs.listFiles(p, false);
while ( fileNames.hasNext() ) {
LocatedFileStatus fileStatus = fileNames.next();
- System.err.println(fileStatus.getPath());
result.add(Path.getPathWithoutSchemeAndAuthority(fileStatus.getPath()).toString());
}
return result;
@@ -615,7 +613,6 @@ public class TestHdfsSpout {
for (int i = 0; i < lineCount; i++) {
os.writeBytes("line " + i + System.lineSeparator());
String msg = "line " + i + System.lineSeparator();
- System.err.print(size + "-" + msg);
size += msg.getBytes().length;
}
os.close();
@@ -660,8 +657,6 @@ public class TestHdfsSpout {
@Override
public List<Integer> emit(String streamId, List<Object> tuple, Object
messageId) {
-// HdfsSpout.MessageId id = (HdfsSpout.MessageId) messageId;
-// lines.add(id.toString() + ' ' + tuple.toString());
lines.add(tuple.toString());
items.add(HdfsUtils.Pair.of(messageId, tuple));
return null;
http://git-wip-us.apache.org/repos/asf/storm/blob/b5240a73/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fed5d3b..34081c8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -213,7 +213,7 @@
<clojure.tools.cli.version>0.2.4</clojure.tools.cli.version>
<disruptor.version>3.3.2</disruptor.version>
<jgrapht.version>0.9.0</jgrapht.version>
- <guava.version>15.0</guava.version>
+ <guava.version>16.0.1</guava.version>
<netty.version>3.9.0.Final</netty.version>
<log4j-over-slf4j.version>1.6.6</log4j-over-slf4j.version>
<log4j.version>2.1</log4j.version>