STORM-3440: client: fix all checkstyle warnings (#3055)
* STORM-3440: client: fix all checkstyle warnings
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
index 34ffd7c..626ac8f 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
@@ -57,21 +57,21 @@
@Override
public Pair<Integer, Integer> apply(Pair<Integer, Integer> sumAndCount, Integer value) {
- return Pair.of(sumAndCount._1 + value, sumAndCount._2 + 1);
+ return Pair.of(sumAndCount.value1 + value, sumAndCount.value2 + 1);
}
@Override
public Pair<Integer, Integer> merge(Pair<Integer, Integer> sumAndCount1, Pair<Integer, Integer> sumAndCount2) {
System.out.println("Merge " + sumAndCount1 + " and " + sumAndCount2);
return Pair.of(
- sumAndCount1._1 + sumAndCount2._1,
- sumAndCount1._2 + sumAndCount2._2
+ sumAndCount1.value1 + sumAndCount2.value1,
+ sumAndCount1.value2 + sumAndCount2.value2
);
}
@Override
public Double result(Pair<Integer, Integer> sumAndCount) {
- return (double) sumAndCount._1 / sumAndCount._2;
+ return (double) sumAndCount.value1 / sumAndCount.value2;
}
}
}
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
index 481758f..e37e666 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
@@ -37,7 +37,7 @@
StreamBuilder builder = new StreamBuilder();
Stream<Tuple3<Integer, Long, Long>> stream = builder.newStream(new RandomIntegerSpout(), TupleValueMappers.of(0, 1, 2));
- PairStream<Long, Integer> pairs = stream.mapToPair(t -> Pair.of(t._2 / 10000, t._1));
+ PairStream<Long, Integer> pairs = stream.mapToPair(t -> Pair.of(t.value2 / 10000, t.value1));
pairs.window(TumblingWindows.of(Count.of(10))).groupByKey().print();
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 9859d08..9c54a19 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
@@ -14,11 +14,11 @@
import org.apache.storm.validation.ConfigValidation.Validator;
import org.apache.storm.validation.ConfigValidationAnnotations.CustomValidator;
-import org.apache.storm.validation.ConfigValidationAnnotations.isBoolean;
-import org.apache.storm.validation.ConfigValidationAnnotations.isInteger;
-import org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryType;
-import org.apache.storm.validation.ConfigValidationAnnotations.isPositiveNumber;
-import org.apache.storm.validation.ConfigValidationAnnotations.isString;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsBoolean;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsInteger;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryType;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsPositiveNumber;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsString;
import org.apache.storm.validation.NotConf;
import org.apache.storm.validation.Validated;
@@ -28,7 +28,7 @@
* @deprecated please use {@link HdfsSpout#setReaderType(String)}
*/
@Deprecated
- @isString
+ @IsString
@CustomValidator(validatorClass = ReaderTypeValidator.class)
public static final String READER_TYPE = "hdfsspout.reader.type";
public static final String TEXT = "text";
@@ -38,67 +38,67 @@
* @deprecated please use {@link HdfsSpout#setHdfsUri(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String HDFS_URI = "hdfsspout.hdfs";
/**
* Required - dir from which to read files.
* @deprecated please use {@link HdfsSpout#setSourceDir(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String SOURCE_DIR = "hdfsspout.source.dir";
/**
* Required - completed files will be moved here.
* @deprecated please use {@link HdfsSpout#setArchiveDir(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String ARCHIVE_DIR = "hdfsspout.archive.dir";
/**
* Required - unparsable files will be moved here.
* @deprecated please use {@link HdfsSpout#setBadFilesDir(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String BAD_DIR = "hdfsspout.badfiles.dir";
/**
* Directory in which lock files will be created.
* @deprecated please use {@link HdfsSpout#setLockDir(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String LOCK_DIR = "hdfsspout.lock.dir";
/**
* Commit after N records. 0 disables this.
* @deprecated please use {@link HdfsSpout#setCommitFrequencyCount(int)}
*/
@Deprecated
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String COMMIT_FREQ_COUNT = "hdfsspout.commit.count";
/**
* Commit after N secs. cannot be disabled.
* @deprecated please use {@link HdfsSpout#setCommitFrequencySec(int)}
*/
@Deprecated
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String COMMIT_FREQ_SEC = "hdfsspout.commit.sec";
/**
* Max outstanding.
* @deprecated please use {@link HdfsSpout#setMaxOutstanding(int)}
*/
@Deprecated
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String MAX_OUTSTANDING = "hdfsspout.max.outstanding";
/**
* Lock timeout.
* @deprecated please use {@link HdfsSpout#setLockTimeoutSec(int)}
*/
@Deprecated
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String LOCK_TIMEOUT = "hdfsspout.lock.timeout.sec";
/**
* If clocks on machines in the Storm cluster are in sync inactivity duration after which locks are considered
@@ -107,14 +107,14 @@
* @deprecated please use {@link HdfsSpout#setClocksInSync(boolean)}
*/
@Deprecated
- @isBoolean
+ @IsBoolean
public static final String CLOCKS_INSYNC = "hdfsspout.clocks.insync";
/**
* Ignore suffix.
* @deprecated please use {@link HdfsSpout#setIgnoreSuffix(String)}
*/
@Deprecated
- @isString
+ @IsString
public static final String IGNORE_SUFFIX = "hdfsspout.ignore.suffix";
/**
* Filenames with this suffix in archive dir will be ignored by the Spout.
@@ -125,7 +125,7 @@
public static final int DEFAULT_COMMIT_FREQ_SEC = 10;
public static final int DEFAULT_MAX_OUTSTANDING = 10000;
public static final int DEFAULT_LOCK_TIMEOUT = 5 * 60; // 5 min
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String DEFAULT_HDFS_CONFIG_KEY = "hdfs.config";
public static class ReaderTypeValidator extends Validator {
diff --git a/storm-client/pom.xml b/storm-client/pom.xml
index 189267e..77c2dd9 100644
--- a/storm-client/pom.xml
+++ b/storm-client/pom.xml
@@ -166,7 +166,7 @@
<!--Note - the version would be inherited-->
<configuration>
<excludes>**/generated/**</excludes>
- <maxAllowedViolations>3067</maxAllowedViolations>
+ <maxAllowedViolations>0</maxAllowedViolations>
</configuration>
</plugin>
<plugin>
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java b/storm-client/src/jvm/org/apache/storm/Config.java
index e86e165..2866b03 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -36,32 +36,33 @@
import org.apache.storm.validation.ConfigValidation.MapOfStringToMapOfStringToObjectValidator;
import org.apache.storm.validation.ConfigValidation.MetricRegistryValidator;
import org.apache.storm.validation.ConfigValidation.MetricReportersValidator;
+import org.apache.storm.validation.ConfigValidationAnnotations;
import org.apache.storm.validation.ConfigValidationAnnotations.CustomValidator;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsBoolean;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsImplementationOfClass;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsInteger;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsKryoReg;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsListEntryCustom;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryCustom;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryType;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsNumber;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsPositiveNumber;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsString;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsStringList;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsStringOrStringList;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsType;
import org.apache.storm.validation.ConfigValidationAnnotations.NotNull;
import org.apache.storm.validation.ConfigValidationAnnotations.Password;
-import org.apache.storm.validation.ConfigValidationAnnotations.isBoolean;
-import org.apache.storm.validation.ConfigValidationAnnotations.isImplementationOfClass;
-import org.apache.storm.validation.ConfigValidationAnnotations.isInteger;
-import org.apache.storm.validation.ConfigValidationAnnotations.isKryoReg;
-import org.apache.storm.validation.ConfigValidationAnnotations.isListEntryCustom;
-import org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryCustom;
-import org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryType;
-import org.apache.storm.validation.ConfigValidationAnnotations.isNumber;
-import org.apache.storm.validation.ConfigValidationAnnotations.isPositiveNumber;
-import org.apache.storm.validation.ConfigValidationAnnotations.isString;
-import org.apache.storm.validation.ConfigValidationAnnotations.isStringList;
-import org.apache.storm.validation.ConfigValidationAnnotations.isStringOrStringList;
-import org.apache.storm.validation.ConfigValidationAnnotations.isType;
/**
* Topology configs are specified as a plain old map. This class provides a convenient way to create a topology config map by providing
* setter methods for all the configs that can be set. It also makes it easier to do things like add serializations.
*
- * This class also provides constants for all the configurations possible on a Storm cluster and Storm topology. Each constant is paired
+ * <p>This class also provides constants for all the configurations possible on a Storm cluster and Storm topology. Each constant is paired
* with an annotation that defines the validity criterion of the corresponding field. Default values for these configs can be found in
* defaults.yaml.
*
- * Note that you may put other configurations in any of the configs. Storm will ignore anything it doesn't recognize, but your topologies
+ * <p>Note that you may put other configurations in any of the configs. Storm will ignore anything it doesn't recognize, but your topologies
* are free to make use of them by reading them in the prepare method of Bolts or the open method of Spouts.
*/
public class Config extends HashMap<String, Object> {
@@ -69,12 +70,12 @@
/**
* The serializer class for ListDelegate (tuple payload). The default serializer will be ListDelegateSerializer
*/
- @isString
+ @IsString
public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
/**
* Disable load aware grouping support.
*/
- @isBoolean
+ @IsBoolean
@NotNull
public static final String TOPOLOGY_DISABLE_LOADAWARE_MESSAGING = "topology.disable.loadaware.messaging";
/**
@@ -82,7 +83,7 @@
* average load is higher than the higher bound, the executor should choose target tasks in a higher scope, The scopes and their orders
* are: EVERYTHING > RACK_LOCAL > HOST_LOCAL > WORKER_LOCAL
*/
- @isPositiveNumber
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_LOCALITYAWARE_HIGHER_BOUND = "topology.localityaware.higher.bound";
/**
@@ -90,14 +91,14 @@
* average load is lower than the lower bound, the executor should choose target tasks in a lower scope. The scopes and their orders
* are: EVERYTHING > RACK_LOCAL > HOST_LOCAL > WORKER_LOCAL
*/
- @isPositiveNumber
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_LOCALITYAWARE_LOWER_BOUND = "topology.localityaware.lower.bound";
/**
* Try to serialize all tuples, even for local transfers. This should only be used for testing, as a sanity check that all of your
* tuples are setup properly.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize";
/**
* A map with blobstore keys mapped to each filename the worker will have access to in the launch directory to the blob by local file
@@ -112,14 +113,14 @@
* How often a worker should check dynamic log level timeouts for expiration. For expired logger settings, the clean up polling task
* will reset the log levels to the original levels (detected at startup), and will clean up the timeout map
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String WORKER_LOG_LEVEL_RESET_POLL_SECS = "worker.log.level.reset.poll.secs";
/**
* How often a task should sync credentials, worst case.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs";
/**
* Whether to enable backpressure in for a certain topology.
@@ -127,58 +128,58 @@
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_BACKPRESSURE_ENABLE = "topology.backpressure.enable";
/**
* A list of users that are allowed to interact with the topology. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_USERS = "topology.users";
/**
* A list of groups that are allowed to interact with the topology. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_GROUPS = "topology.groups";
/**
* A list of readonly users that are allowed to interact with the topology. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_READONLY_USERS = "topology.readonly.users";
/**
* A list of readonly groups that are allowed to interact with the topology. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_READONLY_GROUPS = "topology.readonly.groups";
/**
* True if Storm should timeout messages or not. Defaults to true. This is meant to be used in unit tests to prevent tuples from being
* accidentally timed out during the test.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts";
/**
* When set to true, Storm will log every message that's emitted.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_DEBUG = "topology.debug";
/**
* User defined version of this topology.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_VERSION = "topology.version";
/**
* The fully qualified name of a {@link ShellLogHandler} to handle output from non-JVM processes e.g.
* "com.mycompany.CustomShellLogHandler". If not provided, org.apache.storm.utils.DefaultLogHandler will be used.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_MULTILANG_LOG_HANDLER = "topology.multilang.log.handler";
/**
* The serializer for communication between shell components and non-JVM processes.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer";
/**
* How many processes should be spawned around the cluster to execute this topology. Each process will execute some number of tasks as
@@ -186,8 +187,8 @@
* tune the performance of a topology. The number of workers will be dynamically calculated when the Resource Aware scheduler is used,
* in which case this parameter will not be honored.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_WORKERS = "topology.workers";
/**
* How many instances to create for a spout/bolt. A task runs on a thread with zero or more other tasks for the same spout/bolt. The
@@ -195,64 +196,64 @@
* a spout/bolt can change over time. This allows a topology to scale to more or less resources without redeploying the topology or
* violating the constraints of Storm (such as a fields grouping guaranteeing that the same value goes to the same task).
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_TASKS = "topology.tasks";
/**
* A map of resources used by each component e.g {"cpu.pcore.percent" : 200.0. "onheap.memory.mb": 256.0, "gpu.count" : 2 }
*/
- @isMapEntryType(keyType = String.class, valueType = Number.class)
+ @IsMapEntryType(keyType = String.class, valueType = Number.class)
public static final String TOPOLOGY_COMPONENT_RESOURCES_MAP = "topology.component.resources.map";
/**
* The maximum amount of memory an instance of a spout/bolt will take on heap. This enables the scheduler to allocate slots on machines
* with enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB = "topology.component.resources.onheap.memory.mb";
/**
* The maximum amount of memory an instance of a spout/bolt will take off heap. This enables the scheduler to allocate slots on machines
* with enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB = "topology.component.resources.offheap.memory.mb";
/**
* The config indicates the percentage of cpu for a core an instance(executor) of a component will use. Assuming the a core value to be
* 100, a value of 10 indicates 10% of the core. The P in PCORE represents the term "physical". A default value will be set for this
* config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
/**
* The maximum amount of memory an instance of an acker will take on heap. This enables the scheduler to allocate slots on machines with
* enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_ACKER_RESOURCES_ONHEAP_MEMORY_MB = "topology.acker.resources.onheap.memory.mb";
/**
* The maximum amount of memory an instance of an acker will take off heap. This enables the scheduler to allocate slots on machines
* with enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_ACKER_RESOURCES_OFFHEAP_MEMORY_MB = "topology.acker.resources.offheap.memory.mb";
/**
* The config indicates the percentage of cpu for a core an instance(executor) of an acker will use. Assuming the a core value to be
* 100, a value of 10 indicates 10% of the core. The P in PCORE represents the term "physical". A default value will be set for this
* config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_ACKER_CPU_PCORE_PERCENT = "topology.acker.cpu.pcore.percent";
/**
* The maximum amount of memory an instance of a metrics consumer will take on heap. This enables the scheduler to allocate slots on
* machines with enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_ONHEAP_MEMORY_MB =
"topology.metrics.consumer.resources.onheap.memory.mb";
/**
* The maximum amount of memory an instance of a metrics consumer will take off heap. This enables the scheduler to allocate slots on
* machines with enough available memory. A default value will be set for this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_METRICS_CONSUMER_RESOURCES_OFFHEAP_MEMORY_MB =
"topology.metrics.consumer.resources.offheap.memory.mb";
/**
@@ -260,39 +261,39 @@
* to be 100, a value of 10 indicates 10% of the core. The P in PCORE represents the term "physical". A default value will be set for
* this config if user does not override
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_METRICS_CONSUMER_CPU_PCORE_PERCENT = "topology.metrics.consumer.cpu.pcore.percent";
/**
* The class name of the {@link org.apache.storm.state.StateProvider} implementation. If not specified defaults to {@link
* org.apache.storm.state.InMemoryKeyValueStateProvider}. This can be overridden at the component level.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_STATE_PROVIDER = "topology.state.provider";
/**
* The configuration specific to the {@link org.apache.storm.state.StateProvider} implementation. This can be overridden at the
* component level. The value and the interpretation of this config is based on the state provider implementation. For e.g. this could
* be just a config file name which contains the config for the state provider implementation.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_STATE_PROVIDER_CONFIG = "topology.state.provider.config";
/**
* Topology configuration to specify the checkpoint interval (in millis) at which the topology state is saved when {@link
* org.apache.storm.topology.IStatefulBolt} bolts are involved.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_STATE_CHECKPOINT_INTERVAL = "topology.state.checkpoint.interval.ms";
/**
* A per topology config that specifies the maximum amount of memory a worker can use for that specific topology.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB = "topology.worker.max.heap.size.mb";
/**
* The strategy to use when scheduling a topology with Resource Aware Scheduler.
*/
@NotNull
- @isString
- //NOTE: @isImplementationOfClass(implementsClass = IStrategy.class) is enforced in DaemonConf, so
+ @IsString
+ //NOTE: @IsImplementationOfClass(implementsClass = IStrategy.class) is enforced in DaemonConf, so
// an error will be thrown by nimbus on topology submission and not by the client prior to submitting
// the topology.
public static final String TOPOLOGY_SCHEDULER_STRATEGY = "topology.scheduler.strategy";
@@ -306,38 +307,38 @@
* Array of components that scheduler should try to place on separate hosts when using the constraint solver strategy or the
* multi-tenant scheduler.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components";
/**
* The maximum number of states that will be searched looking for a solution in the constraint solver strategy.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_RAS_CONSTRAINT_MAX_STATE_SEARCH = "topology.ras.constraint.max.state.search";
/**
* The maximum number of states that will be searched looking for a solution in the constraint solver strategy.
* Backward compatibility config value for old topologies
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_RAS_CONSTRAINT_MAX_STATE_TRAVERSAL = "topology.ras.constraint.max.state.traversal";
/**
* The maximum number of seconds to spend scheduling a topology using the constraint solver. Null means no limit.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_RAS_CONSTRAINT_MAX_TIME_SECS = "topology.ras.constraint.max.time.secs";
/**
* A list of host names that this topology would prefer to be scheduled on (no guarantee is given though). This is intended for
* debugging only.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_SCHEDULER_FAVORED_NODES = "topology.scheduler.favored.nodes";
/**
* A list of host names that this topology would prefer to NOT be scheduled on (no guarantee is given though). This is intended for
* debugging only.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_SCHEDULER_UNFAVORED_NODES = "topology.scheduler.unfavored.nodes";
/**
* How many executors to spawn for ackers.
@@ -347,18 +348,18 @@
* If this variable is set to 0, then Storm will immediately ack tuples as soon as they come off the spout,
* effectively disabling reliability.</p>
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors";
/**
* A list of classes implementing IEventLogger (See storm.yaml.example for exact config format). Each listed class will be routed all
* the events sampled from emitting tuples. If there's no class provided to the option, default event logger will be initialized and
* used unless you disable event logger executor.
*
- * Note that EventLoggerBolt takes care of all the implementations of IEventLogger, hence registering many implementations (especially
- * they're implemented as 'blocking' manner) would slow down overall topology.
+ * <p>Note that EventLoggerBolt takes care of all the implementations of IEventLogger, hence registering many
+ * implementations (especially they're implemented as 'blocking' manner) would slow down overall topology.
*/
- @isListEntryCustom(entryValidatorClasses = { EventLoggerRegistryValidator.class })
+ @IsListEntryCustom(entryValidatorClasses = { EventLoggerRegistryValidator.class })
public static final String TOPOLOGY_EVENT_LOGGER_REGISTER = "topology.event.logger.register";
/**
* How many executors to spawn for event logger.
@@ -367,15 +368,15 @@
* configured for this topology (or the estimated number of workers if the Resource Aware Scheduler is used).
* If this variable is set to 0, event logging will be disabled.</p>
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_EVENTLOGGER_EXECUTORS = "topology.eventlogger.executors";
/**
* The maximum amount of time given to the topology to fully process a message emitted by a spout. If the message is not acked within
* this time frame, Storm will fail the message on the spout. Some spouts implementations will then replay the message at a later time.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs";
/**
@@ -383,23 +384,23 @@
* for Storm. A serialization can either be the name of a class (in which case Kryo will automatically create a serializer for the class
* that saves all the object's fields), or an implementation of com.esotericsoftware.kryo.Serializer.
*
- * See Kryo's documentation for more information about writing custom serializers.
+ * <p>See Kryo's documentation for more information about writing custom serializers.
*/
- @isKryoReg
+ @IsKryoReg
public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register";
/**
* A list of classes that customize storm's kryo instance during start-up. Each listed class name must implement IKryoDecorator. During
* start-up the listed class is instantiated with 0 arguments, then its 'decorate' method is called with storm's kryo instance as the
* only argument.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
/**
* Class that specifies how to create a Kryo instance for serialization. Storm will then apply topology.kryo.register and
* topology.kryo.decorators on top of this. The default implementation implements topology.fall.back.on.java.serialization and turns
* references off.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory";
/**
* Whether or not Storm should skip the loading of kryo registrations for which it does not know the class or have the serializer
@@ -409,12 +410,12 @@
* other apps. By setting this config to true, Storm will ignore that it doesn't have those other serializations rather than throw an
* error.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS = "topology.skip.missing.kryo.registrations";
/**
* List of classes to register during state serialization.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_STATE_KRYO_REGISTER = "topology.state.kryo.register";
/**
* A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format). Each listed class will be routed
@@ -422,198 +423,198 @@
* it's parallelism is configurable.
*/
- @isListEntryCustom(entryValidatorClasses = { MetricRegistryValidator.class })
+ @IsListEntryCustom(entryValidatorClasses = { MetricRegistryValidator.class })
public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
/**
* Enable tracking of network message byte counts per source-destination task. This is off by default as it creates tasks^2 metric
* values, but is useful for debugging as it exposes data skew when tuple sizes are uneven.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_SERIALIZED_MESSAGE_SIZE_METRICS = "topology.serialized.message.size.metrics";
/**
* A map of metric name to class name implementing IMetric that will be created once per worker JVM.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String TOPOLOGY_WORKER_METRICS = "topology.worker.metrics";
/**
* A map of metric name to class name implementing IMetric that will be created once per worker JVM.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String WORKER_METRICS = "worker.metrics";
/**
* The maximum parallelism allowed for a component in this topology. This configuration is typically used in testing to limit the number
* of threads spawned in local mode.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_MAX_TASK_PARALLELISM = "topology.max.task.parallelism";
/**
* The maximum number of tuples that can be pending on a spout task at any given time. This config applies to individual tasks, not to
* spouts or topologies as a whole.
*
- * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet. Note that this config parameter has
- * no effect for unreliable spouts that don't tag their tuples with a message id.
+ * <p>A pending tuple is one that has been emitted from a spout but has not been acked or failed yet. Note that this
+ * config parameter has no effect for unreliable spouts that don't tag their tuples with a message id.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_MAX_SPOUT_PENDING = "topology.max.spout.pending";
/**
* The amount of milliseconds the SleepEmptyEmitStrategy should sleep for.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS = "topology.sleep.spout.wait.strategy.time.ms";
/**
* The maximum amount of time a component gives a source of state to synchronize before it requests synchronization again.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS = "topology.state.synchronization.timeout.secs";
/**
* The percentage of tuples to sample to produce stats for a task.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String TOPOLOGY_STATS_SAMPLE_RATE = "topology.stats.sample.rate";
/**
* The time period that builtin metrics data in bucketed into.
*/
- @isInteger
+ @IsInteger
public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS = "topology.builtin.metrics.bucket.size.secs";
/**
* Whether or not to use Java serialization in a topology.
*/
- @isBoolean
+ @IsBoolean
public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION = "topology.fall.back.on.java.serialization";
/**
* Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String TOPOLOGY_WORKER_CHILDOPTS = "topology.worker.childopts";
/**
* Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String TOPOLOGY_WORKER_GC_CHILDOPTS = "topology.worker.gc.childopts";
/**
* Topology-specific options for the logwriter process of a worker.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS = "topology.worker.logwriter.childopts";
/**
* Topology-specific classpath for the worker child process. This is combined to the usual classpath.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String TOPOLOGY_CLASSPATH = "topology.classpath";
/**
* Topology-specific classpath for the worker child process. This will be *prepended* to the usual classpath, meaning it can override
* the Storm classpath. This is for debugging purposes, and is disabled by default. To allow topologies to be submitted with user-first
* classpaths, set the storm.topology.classpath.beginning.enabled config to true.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String TOPOLOGY_CLASSPATH_BEGINNING = "topology.classpath.beginning";
/**
* Topology-specific environment variables for the worker child process. This is added to the existing environment (that of the
* supervisor)
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String TOPOLOGY_ENVIRONMENT = "topology.environment";
/*
* Bolt-specific configuration for windowed bolts to specify the window length as a count of number of tuples
* in the window.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT = "topology.bolts.window.length.count";
/*
* Bolt-specific configuration for windowed bolts to specify the window length in time duration.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS = "topology.bolts.window.length.duration.ms";
/*
* Bolt-specific configuration for windowed bolts to specify the sliding interval as a count of number of tuples.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT = "topology.bolts.window.sliding.interval.count";
/*
* Bolt-specific configuration for windowed bolts to specify the sliding interval in time duration.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS = "topology.bolts.window.sliding.interval.duration.ms";
/**
* Bolt-specific configuration for windowed bolts to specify the name of the stream on which late tuples are going to be emitted. This
* configuration should only be used from the BaseWindowedBolt.withLateTupleStream builder method, and not as global parameter,
* otherwise IllegalArgumentException is going to be thrown.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_BOLTS_LATE_TUPLE_STREAM = "topology.bolts.late.tuple.stream";
/**
* Bolt-specific configuration for windowed bolts to specify the maximum time lag of the tuple timestamp in milliseconds. It means that
* the tuple timestamps cannot be out of order by more than this amount. This config will be effective only if {@link
* org.apache.storm.windowing.TimestampExtractor} is specified.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_TUPLE_TIMESTAMP_MAX_LAG_MS = "topology.bolts.tuple.timestamp.max.lag.ms";
/*
* Bolt-specific configuration for windowed bolts to specify the time interval for generating
* watermark events. Watermark event tracks the progress of time when tuple timestamp is used.
* This config is effective only if {@link org.apache.storm.windowing.TimestampExtractor} is specified.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_BOLTS_WATERMARK_EVENT_INTERVAL_MS = "topology.bolts.watermark.event.interval.ms";
/*
* Bolt-specific configuration for windowed bolts to specify the name of the field in the tuple that holds
* the message id. This is used to track the windowing boundaries and avoid re-evaluating the windows
* during recovery of IStatefulWindowedBolt
*/
- @isString
+ @IsString
public static final String TOPOLOGY_BOLTS_MESSAGE_ID_FIELD_NAME = "topology.bolts.message.id.field.name";
/**
* This config is available for TransactionalSpouts, and contains the id ( a String) for the transactional topology. This id is used to
* store the state of the transactional topology in Zookeeper.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_TRANSACTIONAL_ID = "topology.transactional.id";
/**
* A list of task hooks that are automatically added to every spout and bolt in the topology. An example of when you'd do this is to add
* a hook that integrates with your internal monitoring system. These hooks are instantiated using the zero-arg constructor.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_AUTO_TASK_HOOKS = "topology.auto.task.hooks";
/**
* The size of the receive queue for each executor.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE = "topology.executor.receive.buffer.size";
/**
* The size of the transfer queue for each worker.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE = "topology.transfer.buffer.size";
/**
* The size of the transfer queue for each worker.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String TOPOLOGY_TRANSFER_BATCH_SIZE = "topology.transfer.batch.size";
/**
* How often a tick tuple from the "__system" component and "__tick" stream should be sent to tasks. Meant to be used as a
* component-specific configuration.
*/
- @isInteger
+ @IsInteger
public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS = "topology.tick.tuple.freq.secs";
/**
* The number of tuples to batch before sending to the destination executor.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_PRODUCER_BATCH_SIZE = "topology.producer.batch.size";
/**
@@ -621,98 +622,98 @@
* prevents OutOfMemoryException that can occur in rare scenarios in the presence of BackPressure. This affects only inter-worker
* messages. Messages originating from within the same worker will not be dropped.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
@NotNull
public static final String TOPOLOGY_EXECUTOR_OVERFLOW_LIMIT = "topology.executor.overflow.limit";
/**
* How often a worker should check and notify upstream workers about its tasks that are no longer experiencing BP and able to receive
- * new messages
+ * new messages.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String TOPOLOGY_BACKPRESSURE_CHECK_MILLIS = "topology.backpressure.check.millis";
/**
* How often to send flush tuple to the executors for flushing out batched events.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
@NotNull
public static final String TOPOLOGY_BATCH_FLUSH_INTERVAL_MILLIS = "topology.batch.flush.interval.millis";
/**
* The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed via the TopologyContext.
*/
- @isInteger
+ @IsInteger
public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE = "topology.worker.shared.thread.pool.size";
/**
* The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, an interval of 10
* seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be reported to Zookeeper per task for every
* 10 second interval of time.
*/
- @isInteger
+ @IsInteger
public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS = "topology.error.throttle.interval.secs";
/**
- * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS
+ * See doc for {@link #TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS}.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL = "topology.max.error.report.per.interval";
/**
* How often a batch can be emitted in a Trident topology.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS = "topology.trident.batch.emit.interval.millis";
/**
* Maximum number of tuples that can be stored inmemory cache in windowing operators for fast access without fetching them from store.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_TRIDENT_WINDOWING_INMEMORY_CACHE_LIMIT = "topology.trident.windowing.cache.tuple.limit";
/**
* The id assigned to a running topology. The id is the storm name with a unique nonce appended.
*/
- @isString
+ @IsString
public static final String STORM_ID = "storm.id";
/**
* Name of the topology. This config is automatically set by Storm when the topology is submitted.
*/
- @isString
- public final static String TOPOLOGY_NAME = "topology.name";
+ @IsString
+ public static final String TOPOLOGY_NAME = "topology.name";
/**
- * The principal who submitted a topology
+ * The principal who submitted a topology.
*/
- @isString
- public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
+ @IsString
+ public static final String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
/**
* The local user name of the user who submitted a topology.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user";
/**
* A list of IAutoCredentials that the topology should load and use.
*/
- @isStringList
+ @IsStringList
public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials";
/**
- * Max pending tuples in one ShellBolt
+ * Max pending tuples in one ShellBolt.
*/
@NotNull
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING = "topology.shellbolt.max.pending";
/**
* How long a subprocess can go without heartbeating before the ShellSpout/ShellBolt tries to suicide itself.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_SUBPROCESS_TIMEOUT_SECS = "topology.subprocess.timeout.secs";
/**
* Topology central logging sensitivity to determine who has access to logs in central logging system. The possible values are: S0 -
* Public (open to all users on grid) S1 - Restricted S2 - Confidential S3 - Secret (default.)
*/
- @isString(acceptedValues = { "S0", "S1", "S2", "S3" })
+ @IsString(acceptedValues = { "S0", "S1", "S2", "S3" })
public static final String TOPOLOGY_LOGGING_SENSITIVITY = "topology.logging.sensitivity";
/**
* Log file the user can use to configure Log4j2.
@@ -721,186 +722,186 @@
* The configs are merged according to the rules here:
* https://logging.apache.org/log4j/2.x/manual/configuration.html#CompositeConfiguration
*/
- @isString
+ @IsString
public static final String TOPOLOGY_LOGGING_CONFIG_FILE = "topology.logging.config";
/**
- * Sets the priority for a topology
+ * Sets the priority for a topology.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_PRIORITY = "topology.priority";
/**
* The root directory in ZooKeeper for metadata about TransactionalSpouts.
*/
- @isString
+ @IsString
public static final String TRANSACTIONAL_ZOOKEEPER_ROOT = "transactional.zookeeper.root";
/**
* The list of zookeeper servers in which to keep the transactional state. If null (which is default), will use storm.zookeeper.servers
*/
- @isStringList
+ @IsStringList
public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS = "transactional.zookeeper.servers";
/**
* The port to use to connect to the transactional zookeeper servers. If null (which is default), will use storm.zookeeper.port
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TRANSACTIONAL_ZOOKEEPER_PORT = "transactional.zookeeper.port";
/**
* The user as which the nimbus client should be acquired to perform the operation.
*/
- @isString
+ @IsString
public static final String STORM_DO_AS_USER = "storm.doAsUser";
/**
* The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler to
* org.apache.storm.scheduler.multitenant.MultitenantScheduler
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
/**
* A class that implements a wait strategy for spout. Waiting is triggered in one of two conditions:
*
- * 1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and can't emit any more tuples
+ * <p>1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and can't emit any more tuples
*
- * This class must implement {@link IWaitStrategy}.
+ * <p>This class must implement {@link IWaitStrategy}.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY = "topology.spout.wait.strategy";
/**
* Configures park time for WaitStrategyPark for spout. If set to 0, returns immediately (i.e busy wait).
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_SPOUT_WAIT_PARK_MICROSEC = "topology.spout.wait.park.microsec";
/**
- * Configures number of iterations to spend in level 1 of WaitStrategyProgressive, before progressing to level 2
+ * Configures number of iterations to spend in level 1 of WaitStrategyProgressive, before progressing to level 2.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL1_COUNT = "topology.spout.wait.progressive.level1.count";
/**
- * Configures number of iterations to spend in level 2 of WaitStrategyProgressive, before progressing to level 3
+ * Configures number of iterations to spend in level 2 of WaitStrategyProgressive, before progressing to level 3.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL2_COUNT = "topology.spout.wait.progressive.level2.count";
/**
* Configures sleep time for WaitStrategyProgressive.
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS = "topology.spout.wait.progressive.level3.sleep.millis";
/**
* Selects the Bolt's Wait Strategy to use when there are no incoming msgs. Used to trade off latency vs CPU usage. This class must
* implement {@link IWaitStrategy}.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_BOLT_WAIT_STRATEGY = "topology.bolt.wait.strategy";
/**
* Configures park time for WaitStrategyPark. If set to 0, returns immediately (i.e busy wait).
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BOLT_WAIT_PARK_MICROSEC = "topology.bolt.wait.park.microsec";
/**
- * Configures number of iterations to spend in level 1 of WaitStrategyProgressive, before progressing to level 2
+ * Configures number of iterations to spend in level 1 of WaitStrategyProgressive, before progressing to level 2.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL1_COUNT = "topology.bolt.wait.progressive.level1.count";
/**
- * Configures number of iterations to spend in level 2 of WaitStrategyProgressive, before progressing to level 3
+ * Configures number of iterations to spend in level 2 of WaitStrategyProgressive, before progressing to level 3.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL2_COUNT = "topology.bolt.wait.progressive.level2.count";
/**
* Configures sleep time for WaitStrategyProgressive.
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS = "topology.bolt.wait.progressive.level3.sleep.millis";
/**
* A class that implements a wait strategy for an upstream component (spout/bolt) trying to write to a downstream component whose recv
* queue is full
*
- * 1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and can't emit any more tuples
+ * <p>1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and can't emit any more tuples
*
- * This class must implement {@link IWaitStrategy}.
+ * <p>This class must implement {@link IWaitStrategy}.
*/
- @isString
+ @IsString
public static final String TOPOLOGY_BACKPRESSURE_WAIT_STRATEGY = "topology.backpressure.wait.strategy";
/**
* Configures park time if using WaitStrategyPark for BackPressure. If set to 0, returns immediately (i.e busy wait).
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BACKPRESSURE_WAIT_PARK_MICROSEC = "topology.backpressure.wait.park.microsec";
/**
* Configures sleep time if using WaitStrategyProgressive for BackPressure.
*/
@NotNull
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS =
"topology.backpressure.wait.progressive.level3.sleep.millis";
/**
* Configures steps used to determine progression to the next level of wait .. if using WaitStrategyProgressive for BackPressure.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL1_COUNT = "topology.backpressure.wait.progressive.level1.count";
/**
* Configures steps used to determine progression to the next level of wait .. if using WaitStrategyProgressive for BackPressure.
*/
@NotNull
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL2_COUNT = "topology.backpressure.wait.progressive.level2.count";
/**
* Check recvQ after every N invocations of Spout's nextTuple() [when ACKing is disabled]. Spouts receive very few msgs if ACK is
* disabled. This avoids checking the recvQ after each nextTuple().
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
@NotNull
public static final String TOPOLOGY_SPOUT_RECVQ_SKIPS = "topology.spout.recvq.skips";
/**
* Minimum number of nimbus hosts where the code must be replicated before leader nimbus is allowed to perform topology activation tasks
* like setting up heartbeats/assignments and marking the topology as active. default is 0.
*/
- @isNumber
+ @IsNumber
public static final String TOPOLOGY_MIN_REPLICATION_COUNT = "topology.min.replication.count";
/**
* Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count. Once this time is elapsed nimbus will
* go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. The default is 0
* seconds, a value of -1 indicates to wait for ever.
*/
- @isNumber
+ @IsNumber
public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "topology.max.replication.wait.time.sec";
/**
* The list of servers that Pacemaker is running on.
*/
- @isStringList
+ @IsStringList
public static final String PACEMAKER_SERVERS = "pacemaker.servers";
/**
* The port Pacemaker should run on. Clients should connect to this port to submit or read heartbeats.
*/
- @isNumber
- @isPositiveNumber
+ @IsNumber
+ @IsPositiveNumber
public static final String PACEMAKER_PORT = "pacemaker.port";
/**
* The maximum number of threads that should be used by the Pacemaker client.
* When Pacemaker gets loaded it will spawn new threads, up to
* this many total, to handle the load.
*/
- @isNumber
- @isPositiveNumber
+ @IsNumber
+ @IsPositiveNumber
public static final String PACEMAKER_CLIENT_MAX_THREADS = "pacemaker.client.max.threads";
/**
* This should be one of "DIGEST", "KERBEROS", or "NONE" Determines the mode of authentication the pacemaker server and client use. The
@@ -913,189 +914,188 @@
/**
* Pacemaker Thrift Max Message Size (bytes).
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String PACEMAKER_THRIFT_MESSAGE_SIZE_MAX = "pacemaker.thrift.message.size.max";
/**
* Max no.of seconds group mapping service will cache user groups
*/
- @isInteger
+ @IsInteger
public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs";
/**
* List of DRPC servers so that the DRPCSpout knows who to talk to.
*/
- @isStringList
+ @IsStringList
public static final String DRPC_SERVERS = "drpc.servers";
/**
* This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
/**
* The number of times to retry a Nimbus operation.
*/
- @isNumber
+ @IsNumber
public static final String STORM_NIMBUS_RETRY_TIMES = "storm.nimbus.retry.times";
/**
* The starting interval between exponential backoff retries of a Nimbus operation.
*/
- @isNumber
+ @IsNumber
public static final String STORM_NIMBUS_RETRY_INTERVAL = "storm.nimbus.retry.interval.millis";
/**
* The ceiling of the interval between retries of a client connect to Nimbus operation.
*/
- @isNumber
+ @IsNumber
public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING = "storm.nimbus.retry.intervalceiling.millis";
/**
- * The Nimbus transport plug-in for Thrift client/server communication
+ * The Nimbus transport plug-in for Thrift client/server communication.
*/
- @isString
+ @IsString
public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport";
/**
* Which port the Thrift interface of Nimbus should run on. Clients should connect to this port to upload jars and submit topologies.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port";
/**
* Nimbus thrift server queue size, default is 100000. This is the request queue size , when there are more requests than number of
* threads to serve the requests, those requests will be queued to this queue. If the request queue size > this config, then the
* incoming requests will be rejected.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_QUEUE_SIZE = "nimbus.queue.size";
/**
* Nimbus assignments backend for storing local assignments. We will use it to store physical plan and runtime storm ids.
*/
- @isString
- @isImplementationOfClass(implementsClass = org.apache.storm.assignments.ILocalAssignmentsBackend.class)
+ @IsString
+ @ConfigValidationAnnotations.IsImplementationOfClass(implementsClass = org.apache.storm.assignments.ILocalAssignmentsBackend.class)
public static final String NIMBUS_LOCAL_ASSIGNMENTS_BACKEND_CLASS = "nimbus.local.assignments.backend.class";
/**
* The number of threads that should be used by the nimbus thrift server.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads";
/**
* The maximum buffer size thrift should use when reading messages.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size";
/**
* How long before a Thrift Client socket hangs before timeout and restart the socket.
*/
- @isInteger
+ @IsInteger
public static final String STORM_THRIFT_SOCKET_TIMEOUT_MS = "storm.thrift.socket.timeout.ms";
/**
- * The DRPC transport plug-in for Thrift client/server communication
+ * The DRPC transport plug-in for Thrift client/server communication.
*/
- @isString
+ @IsString
public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";
/**
* This port is used by Storm DRPC for receiving DPRC requests from clients.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String DRPC_PORT = "drpc.port";
/**
- * DRPC thrift server queue size
+ * DRPC thrift server queue size.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
/**
- * DRPC thrift server worker threads
+ * DRPC thrift server worker threads.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
/**
* The maximum buffer size thrift should use when reading messages for DRPC.
*/
- @isNumber
- @isPositiveNumber
+ @IsNumber
+ @IsPositiveNumber
public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size";
/**
- * The DRPC invocations transport plug-in for Thrift client/server communication
+ * The DRPC invocations transport plug-in for Thrift client/server communication.
*/
- @isString
+ @IsString
public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport";
/**
- * DRPC invocations thrift server worker threads
+ * DRPC invocations thrift server worker threads.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads";
/**
* Initialization parameters for the group mapping service plugin. Provides a way for a
- * @link{STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN}
- * implementation to access optional settings.
+ * {@link #STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN} implementation to access optional settings.
*/
- @isType(type = Map.class)
+ @IsType(type = Map.class)
public static final String STORM_GROUP_MAPPING_SERVICE_PARAMS = "storm.group.mapping.service.params";
/**
- * The default transport plug-in for Thrift client/server communication
+ * The default transport plug-in for Thrift client/server communication.
*/
- @isString
+ @IsString
public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport";
/**
* How long a worker can go without heartbeating before the supervisor tries to restart the worker process.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
/**
* How many seconds to allow for graceful worker shutdown when killing workers before resorting to force kill.
* If a worker fails to shut down gracefully within this delay, it will either suicide or be forcibly killed by the supervisor.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS = "supervisor.worker.shutdown.sleep.secs";
/**
* A list of hosts of ZooKeeper servers used to manage the cluster.
*/
- @isStringList
+ @IsStringList
public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers";
/**
* The port Storm will use to connect to each of the ZooKeeper servers.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port";
/**
* This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for the user Nimbus and Supervisors use to authenticate
* with ZK.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL";
/**
* The ACL of the drpc user in zookeeper so the drpc servers can verify worker tokens.
*
- * Should be in the form 'scheme:acl' just like STORM_ZOOKEEPER_SUPERACL.
+ * <p>Should be in the form 'scheme:acl' just like STORM_ZOOKEEPER_SUPERACL.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_DRPC_ACL = "storm.zookeeper.drpcACL";
/**
* The topology Zookeeper authentication scheme to use, e.g. "digest". It is the internal config and user shouldn't set it.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME = "storm.zookeeper.topology.auth.scheme";
/**
* The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk. This is NOT used for
* compressing serialized tuples sent between topologies.
*/
- @isString
+ @IsString
public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate";
- @isListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class })
+ @IsListEntryCustom(entryValidatorClasses = { MetricReportersValidator.class })
public static final String STORM_METRICS_REPORTERS = "storm.metrics.reporters";
/**
* What blobstore implementation the storm client should use.
*/
- @isString
+ @IsString
public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
/**
@@ -1103,81 +1103,82 @@
* LocalFsBlobStore it could be either absolute or relative. If the setting is a relative directory, it is relative to root directory of
* Storm installation.
*/
- @isString
+ @IsString
public static final String BLOBSTORE_DIR = "blobstore.dir";
/**
* Enable the blobstore cleaner. Certain blobstores may only want to run the cleaner on one daemon. Currently Nimbus handles setting
* this.
*/
- @isBoolean
+ @IsBoolean
public static final String BLOBSTORE_CLEANUP_ENABLE = "blobstore.cleanup.enable";
/**
* principal for nimbus/supervisor to use to access secure hdfs for the blobstore.
* The format is generally "primary/instance@REALM", where "instance" field is optional.
* If the instance field of the principal is the string "_HOST", it will
- + be replaced with the host name of the server the daemon is running on (by calling {@link #getBlobstoreHDFSPrincipal(Map conf)} method).
+ * be replaced with the host name of the server the daemon is running on (by calling
+ * {@link #getBlobstoreHDFSPrincipal(Map conf)} method).
*/
- @isString
+ @IsString
public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal";
/**
* keytab for nimbus/supervisor to use to access secure hdfs for the blobstore.
*/
- @isString
+ @IsString
public static final String BLOBSTORE_HDFS_KEYTAB = "blobstore.hdfs.keytab";
/**
- * Set replication factor for a blob in HDFS Blobstore Implementation
+ * Set replication factor for a blob in HDFS Blobstore Implementation.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String STORM_BLOBSTORE_REPLICATION_FACTOR = "storm.blobstore.replication.factor";
/**
* The hostname the supervisors/workers should report to nimbus. If unset, Storm will get the hostname to report by calling
* <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
*
- * You should set this config when you don't have a DNS which supervisors/workers can utilize to find each other based on hostname got
- * from calls to
+ * <p>You should set this config when you don't have a DNS which supervisors/workers can utilize to find each other
+ * based on hostname got from calls to
* <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
*/
- @isString
+ @IsString
public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname";
/**
* List of seed nimbus hosts to use for leader nimbus discovery.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_SEEDS = "nimbus.seeds";
/**
* A list of users that are the only ones allowed to run user operation on storm cluster. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_USERS = "nimbus.users";
/**
* A list of groups , users belong to these groups are the only ones allowed to run user operation on storm cluster. To use this set
* nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_GROUPS = "nimbus.groups";
/**
* The mode this Storm cluster is running in. Either "distributed" or "local".
*/
- @isString
+ @IsString
public static final String STORM_CLUSTER_MODE = "storm.cluster.mode";
/**
* The root location at which Storm stores data in ZooKeeper.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root";
/**
* A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during
* authentication.
*/
- @isString
+ @IsString
@Password
public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD = "storm.zookeeper.topology.auth.payload";
/**
* The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_AUTH_SCHEME = "storm.zookeeper.auth.scheme";
/**
* A string representing the payload for cluster Zookeeper authentication. It gets serialized using UTF-8 encoding during
@@ -1185,7 +1186,7 @@
* in the storm-cluster-auth.yaml file. This file storm-cluster-auth.yaml should then be protected with appropriate permissions that
* deny access from workers.
*/
- @isString
+ @IsString
public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD = "storm.zookeeper.auth.payload";
/**
* What Network Topography detection classes should we use. Given a list of supervisor hostnames (or IP addresses), this class would
@@ -1193,101 +1194,101 @@
* resource aware scheduler.
*/
@NotNull
- @isImplementationOfClass(implementsClass = org.apache.storm.networktopography.DNSToSwitchMapping.class)
+ @IsImplementationOfClass(implementsClass = org.apache.storm.networktopography.DNSToSwitchMapping.class)
public static final String STORM_NETWORK_TOPOGRAPHY_PLUGIN = "storm.network.topography.plugin";
/**
* The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced with an identifier for this
* worker. Because the JVM complains about multiple GC opts the topology can override this default value by setting
* topology.worker.gc.childopts.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts";
/**
* The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%", "%WORKER-PORT%" and
* "%HEAP-MEM%" substrings are replaced with: %ID% -> port (for backward compatibility), %WORKER-ID% -> worker-id,
* %TOPOLOGY-ID% -> topology-id, %WORKER-PORT% -> port. %HEAP-MEM% -> mem-onheap.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String WORKER_CHILDOPTS = "worker.childopts";
/**
- * The default heap memory size in MB per worker, used in the jvm -Xmx opts for launching the worker
+ * The default heap memory size in MB per worker, used in the jvm -Xmx opts for launching the worker.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String WORKER_HEAP_MEMORY_MB = "worker.heap.memory.mb";
/**
* The total amount of memory (in MiB) a supervisor is allowed to give to its workers. A default value will be set for this config if
* user does not override
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String SUPERVISOR_MEMORY_CAPACITY_MB = "supervisor.memory.capacity.mb";
/**
* The total amount of CPU resources a supervisor is allowed to give to its workers. By convention 1 cpu core should be about 100, but
* this can be adjusted if needed using 100 makes it simple to set the desired value to the capacity measurement for single threaded
* bolts. A default value will be set for this config if user does not override
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
/**
* Port used for supervisor thrift server.
*/
public static final String SUPERVISOR_THRIFT_PORT = "supervisor.thrift.port";
- @isString
+ @IsString
/**
* The Supervisor invocations transport plug-in for Thrift client/server communication.
*/
public static final String SUPERVISOR_THRIFT_TRANSPORT_PLUGIN = "supervisor.thrift.transport";
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
/**
* Supervisor thrift server queue size.
*/
public static final String SUPERVISOR_QUEUE_SIZE = "supervisor.queue.size";
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
/**
* The number of threads that should be used by the supervisor thrift server.
*/
public static final String SUPERVISOR_THRIFT_THREADS = "supervisor.thrift.threads";
- @isNumber
- @isPositiveNumber
+ @IsNumber
+ @IsPositiveNumber
public static final String SUPERVISOR_THRIFT_MAX_BUFFER_SIZE = "supervisor.thrift.max_buffer_size";
/**
* How long before a supervisor Thrift Client socket hangs before timeout and restart the socket.
*/
- @isInteger
+ @IsInteger
public static final String SUPERVISOR_THRIFT_SOCKET_TIMEOUT_MS = "supervisor.thrift.socket.timeout.ms";
/**
* A map of resources the Supervisor has e.g {"cpu.pcore.percent" : 200.0. "onheap.memory.mb": 256.0, "gpu.count" : 2.0 }
*/
- @isMapEntryType(keyType = String.class, valueType = Number.class)
+ @IsMapEntryType(keyType = String.class, valueType = Number.class)
public static final String SUPERVISOR_RESOURCES_MAP = "supervisor.resources.map";
/**
* Whether or not to use ZeroMQ for messaging in local mode. If this is set to false, then Storm will use a pure-Java messaging system.
* The purpose of this flag is to make it easy to run Storm in local mode by eliminating the need for native dependencies, which can be
* difficult to install.
*
- * Defaults to false.
+ * <p>Defaults to false.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq";
/**
- * The transporter for communication among Storm tasks
+ * The transporter for communication among Storm tasks.
*/
- @isString
+ @IsString
public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
/**
* Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication";
/**
- * Netty based messaging: The buffer size for send/recv buffer
+ * Netty based messaging: The buffer size for send/recv buffer.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
/**
* Netty based messaging: The netty write buffer high watermark in bytes.
@@ -1297,8 +1298,8 @@
* low water mark}.
* </p>
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_MESSAGING_NETTY_BUFFER_HIGH_WATERMARK = "storm.messaging.netty.buffer.high.watermark";
/**
* Netty based messaging: The netty write buffer low watermark in bytes.
@@ -1307,299 +1308,301 @@
* mark} and then dropped down below this value, the netty {@code Channel.isWritable()} will start to return true.
* </p>
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_MESSAGING_NETTY_BUFFER_LOW_WATERMARK = "storm.messaging.netty.buffer.low.watermark";
/**
- * Netty based messaging: Sets the backlog value to specify when the channel binds to a local address
+ * Netty based messaging: Sets the backlog value to specify when the channel binds to a local address.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_MESSAGING_NETTY_SOCKET_BACKLOG = "storm.messaging.netty.socket.backlog";
/**
* Netty based messaging: The # of worker threads for the server.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads";
/**
* If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of
- * STORM_NETTY_MESSAGE_BATCH_SIZE bytes
+ * STORM_NETTY_MESSAGE_BATCH_SIZE bytes.
*/
- @isInteger
+ @IsInteger
public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size";
/**
* Netty based messaging: The min # of milliseconds that a peer will wait.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
/**
* Netty based messaging: The max # of milliseconds that a peer will wait.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
/**
* Netty based messaging: The # of worker threads for the client.
*/
- @isInteger
+ @IsInteger
public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads";
/**
* Should the supervior try to run the worker as the lauching user or not. Defaults to false.
*/
- @isBoolean
+ @IsBoolean
public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user";
/**
- * max timeout for supervisor reported heartbeats when master gains leadership
+ * max timeout for supervisor reported heartbeats when master gains leadership.
*/
- @isInteger
+ @IsInteger
public static final String SUPERVISOR_WORKER_HEARTBEATS_MAX_TIMEOUT_SECS = "supervisor.worker.heartbeats.max.timeout.secs";
/**
* On some systems (windows for example) symlinks require special privileges that not everyone wants to grant a headless user. You can
* completely disable the use of symlinks by setting this config to true, but by doing so you may also lose some features from storm.
* For example the blobstore feature does not currently work without symlinks enabled.
*/
- @isBoolean
+ @IsBoolean
public static final String DISABLE_SYMLINKS = "storm.disable.symlinks";
/**
* The plugin that will convert a principal to a local user.
*/
- @isString
+ @IsString
public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal";
/**
- * The plugin that will provide user groups service
+ * The plugin that will provide user groups service.
*/
- @isString
+ @IsString
public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service";
/**
* A list of credential renewers that nimbus should load.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes";
/**
* A list of plugins that nimbus should load during submit topology to populate credentials on user's behalf.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes";
/**
* A list of users that run the supervisors and should be authorized to interact with nimbus as a supervisor would. To use this set
* nimbus.authorizer to org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users";
/**
* A list of users that nimbus runs as and should be authorized to interact with the supervisor as nimbus would. To use this set
* supervisor.authorizer to org.apache.storm.security.auth.authorizer.SupervisorSimpleACLAuthorizer.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_DAEMON_USERS = "nimbus.daemon.users";
/**
* A list of users that are cluster admins and can run any command. To use this set nimbus.authorizer to
* org.apache.storm.security.auth.authorizer.SimpleACLAuthorizer
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_ADMINS = "nimbus.admins";
/**
* A list of groups that are cluster admins and can run any command.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_ADMINS_GROUPS = "nimbus.admins.groups";
/**
- * For secure mode we would want to turn on this config By default this is turned off assuming the default is insecure
+ * For secure mode we would want to turn on this config By default this is turned off assuming the default is insecure.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_BLOBSTORE_ACL_VALIDATION_ENABLED = "storm.blobstore.acl.validation.enabled";
/**
* What buffer size to use for the blobstore uploads.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
/**
* What chuck size to use for storm client to upload dependency jars.
*/
- @isPositiveNumber
- @isInteger
- public static final String STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUCK_SIZE_BYTES = "storm.blobstore.dependency.jar.upload.chuck.size.bytes";
+ @IsPositiveNumber
+ @IsInteger
+ public static final String STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUCK_SIZE_BYTES =
+ "storm.blobstore.dependency.jar.upload.chuck.size.bytes";
/**
* FQCN of a class that implements {@code ISubmitterHook} @see ISubmitterHook for details.
*/
- @isString
+ @IsString
public static final String STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN = "storm.topology.submission.notifier.plugin.class";
/**
* Impersonation user ACL config entries.
*/
- @isMapEntryCustom(keyValidatorClasses = { ConfigValidation.StringValidator.class },
+ @IsMapEntryCustom(keyValidatorClasses = { ConfigValidation.StringValidator.class },
valueValidatorClasses = { ConfigValidation.ImpersonationAclUserEntryValidator.class })
public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl";
/**
* A whitelist of the RAS scheduler strategies allowed by nimbus. Should be a list of fully-qualified class names or null to allow all.
*/
- @isStringList
+ @IsStringList
public static final String NIMBUS_SCHEDULER_STRATEGY_CLASS_WHITELIST = "nimbus.scheduler.strategy.class.whitelist";
/**
* Full path to the worker-laucher executable that will be used to lauch workers when SUPERVISOR_RUN_WORKER_AS_USER is set to true.
*/
- @isString
+ @IsString
public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher";
/**
* Map a version of storm to a worker classpath that can be used to run it. This allows the supervisor to select an available version of
* storm that is compatible with what a topology was launched with.
*
- * Only the major and minor version numbers are used, although this may change in the future. The code will first try to find a version
+ * <p>Only the major and minor version numbers are used, although this may change in the future. The code will
+ * first try to find a version
* that is the same or higher than the requested version, but with the same major version number. If it cannot it will fall back to
* using one with a lower minor version, but in some cases this might fail as some features may be missing.
*
- * Because of how this selection process works please don't include two releases with the same major and minor versions as it is
+ * <p>Because of how this selection process works please don't include two releases with the same major and minor versions as it is
* undefined which will be selected. Also it is good practice to just include one release for each major version you want to support
* unless the minor versions are truly not compatible with each other. This is to avoid maintenance and testing overhead.
*
- * This config needs to be set on all supervisors and on nimbus. In general this can be the output of calling storm classpath on the
+ * <p>This config needs to be set on all supervisors and on nimbus. In general this can be the output of calling storm classpath on the
* version you want and adding in an entry for the config directory for that release. You should modify the storm.yaml of each of these
* versions to match the features and settings you want on the main version.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP = "supervisor.worker.version.classpath.map";
/**
* Map a version of storm to a worker's main class. In most cases storm should have correct defaults and just setting
* SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP is enough.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String SUPERVISOR_WORKER_VERSION_MAIN_MAP = "supervisor.worker.version.main.map";
/**
* Map a version of storm to a worker's logwriter class. In most cases storm should have correct defaults and just setting
* SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP is enough.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String SUPERVISOR_WORKER_VERSION_LOGWRITER_MAP = "supervisor.worker.version.logwriter.map";
/**
* The version of storm to assume a topology should run as if not version is given by the client when submitting the topology.
*/
- @isString
+ @IsString
public static final String SUPERVISOR_WORKER_DEFAULT_VERSION = "supervisor.worker.default.version";
/**
* A directory on the local filesystem used by Storm for any local filesystem usage it needs. The directory must exist and the Storm
* daemons must have permission to read/write from this location. It could be either absolute or relative. If the setting is a relative
* directory, it is relative to root directory of Storm installation.
*/
- @isString
+ @IsString
public static final String STORM_LOCAL_DIR = "storm.local.dir";
/**
* The workers-artifacts directory (where we place all workers' logs), can be either absolute or relative. By default,
* ${storm.log.dir}/workers-artifacts is where worker logs go. If the setting is a relative directory, it is relative to storm.log.dir.
*/
- @isString
+ @IsString
public static final String STORM_WORKERS_ARTIFACTS_DIR = "storm.workers.artifacts.dir";
/**
* A list of hosts of Exhibitor servers used to discover/maintain connection to ZooKeeper cluster. Any configured ZooKeeper servers will
* be used for the curator/exhibitor backup connection string.
*/
- @isStringList
+ @IsStringList
public static final String STORM_EXHIBITOR_SERVERS = "storm.exhibitor.servers";
/**
* The port Storm will use to connect to each of the exhibitor servers.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String STORM_EXHIBITOR_PORT = "storm.exhibitor.port";
/*
* How often to poll Exhibitor cluster in millis.
*/
- @isString
+ @IsString
public static final String STORM_EXHIBITOR_URIPATH = "storm.exhibitor.poll.uripath";
/**
* How often to poll Exhibitor cluster in millis.
*/
- @isInteger
+ @IsInteger
public static final String STORM_EXHIBITOR_POLL = "storm.exhibitor.poll.millis";
/**
* The number of times to retry an Exhibitor operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_EXHIBITOR_RETRY_TIMES = "storm.exhibitor.retry.times";
/*
* The interval between retries of an Exhibitor operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_EXHIBITOR_RETRY_INTERVAL = "storm.exhibitor.retry.interval";
/**
* The ceiling of the interval between retries of an Exhibitor operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_EXHIBITOR_RETRY_INTERVAL_CEILING = "storm.exhibitor.retry.intervalceiling.millis";
/**
* The connection timeout for clients to ZooKeeper.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout";
/**
* The session timeout for clients to ZooKeeper.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout";
/**
* The interval between retries of a Zookeeper operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ZOOKEEPER_RETRY_INTERVAL = "storm.zookeeper.retry.interval";
/**
* The ceiling of the interval between retries of a Zookeeper operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING = "storm.zookeeper.retry.intervalceiling.millis";
/**
* The number of times to retry a Zookeeper operation.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ZOOKEEPER_RETRY_TIMES = "storm.zookeeper.retry.times";
/**
* The ClusterState factory that worker will use to create a ClusterState to store state in. Defaults to ZooKeeper.
*/
- @isString
+ @IsString
public static final String STORM_CLUSTER_STATE_STORE = "storm.cluster.state.store";
/**
* How often this worker should heartbeat to the supervisor.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";
/**
* How often executor metrics should report to master, used for RPC heartbeat mode.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String EXECUTOR_METRICS_FREQUENCY_SECS = "executor.metrics.frequency.secs";
/**
* How often a task should heartbeat its status to the master, deprecated for 2.0 RPC heartbeat reporting, see {@code
* EXECUTOR_METRICS_FREQUENCY_SECS }.
*/
@Deprecated
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs";
/**
* How often a task should sync its connections with other tasks (if a task is reassigned, the other tasks sending messages to it need
* to refresh their connections). In general though, when a reassignment happens other tasks will be notified almost immediately. This
* configuration is here just in case that notification doesn't come through.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs";
/**
* The Access Control List for the DRPC Authorizer.
*
* @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
*/
- @isType(type = Map.class)
+ @IsType(type = Map.class)
public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl";
/**
* File name of the DRPC Authorizer ACL.
*
* @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
*/
- @isString
+ @IsString
public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename";
/**
* Whether the DRPCSimpleAclAuthorizer should deny requests for operations involving functions that have no explicit ACL entry. When set
@@ -1609,39 +1612,39 @@
*
* @see org.apache.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
*/
- @isBoolean
+ @IsBoolean
public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict";
/**
- * root directory of the storm cgroup hierarchy
+ * root directory of the storm cgroup hierarchy.
*/
- @isString
+ @IsString
public static final String STORM_CGROUP_HIERARCHY_DIR = "storm.cgroup.hierarchy.dir";
/**
- * The number of Buckets
+ * The number of Buckets.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NUM_STAT_BUCKETS = "num.stat.buckets";
/**
* Interval to check for the worker to check for updated blobs and refresh worker state accordingly. The default is 10 seconds
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String WORKER_BLOB_UPDATE_POLL_INTERVAL_SECS = "worker.blob.update.poll.interval.secs";
/**
* A specify Locale for daemon metrics reporter plugin. Use the specified IETF BCP 47 language tag string for a Locale.
*/
- @isString
+ @IsString
public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_LOCALE = "storm.daemon.metrics.reporter.plugin.locale";
/**
* A specify rate-unit in TimeUnit to specify reporting frequency for daemon metrics reporter plugin.
*/
- @isString
+ @IsString
public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_RATE_UNIT = "storm.daemon.metrics.reporter.plugin.rate.unit";
/**
* A specify duration-unit in TimeUnit to specify reporting window for daemon metrics reporter plugin.
*/
- @isString
+ @IsString
public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_DURATION_UNIT = "storm.daemon.metrics.reporter.plugin.duration.unit";
//DO NOT CHANGE UNLESS WE ADD IN STATE NOT STORED IN THE PARENT CLASS
private static final long serialVersionUID = -1550278723792864455L;
@@ -1778,38 +1781,47 @@
return ret;
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setClasspath(String cp) {
setClasspath(this, cp);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setEnvironment(Map<String, Object> env) {
setEnvironment(this, env);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setDebug(boolean isOn) {
setDebug(this, isOn);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setTopologyVersion(String version) {
setTopologyVersion(this, version);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setNumWorkers(int workers) {
setNumWorkers(this, workers);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setNumAckers(int numExecutors) {
setNumAckers(this, numExecutors);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setNumEventLoggers(int numExecutors) {
setNumEventLoggers(this, numExecutors);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setMessageTimeoutSecs(int secs) {
setMessageTimeoutSecs(this, secs);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void registerSerialization(Class klass) {
registerSerialization(this, klass);
}
@@ -1818,6 +1830,7 @@
registerSerialization(this, klass, serializerClass);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void registerEventLogger(Class<? extends IEventLogger> klass, Map<String, Object> argument) {
registerEventLogger(this, klass, argument);
}
@@ -1826,6 +1839,7 @@
registerEventLogger(this, klass, null);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
registerMetricsConsumer(this, klass, argument, parallelismHint);
}
@@ -1838,30 +1852,37 @@
registerMetricsConsumer(this, klass);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void registerDecorator(Class<? extends IKryoDecorator> klass) {
registerDecorator(this, klass);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setKryoFactory(Class<? extends IKryoFactory> klass) {
setKryoFactory(this, klass);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setSkipMissingKryoRegistrations(boolean skip) {
setSkipMissingKryoRegistrations(this, skip);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setMaxTaskParallelism(int max) {
setMaxTaskParallelism(this, max);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setMaxSpoutPending(int max) {
setMaxSpoutPending(this, max);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setStatsSampleRate(double rate) {
setStatsSampleRate(this, rate);
}
+ @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
public void setFallBackOnJavaSerialization(boolean fallback) {
setFallBackOnJavaSerialization(this, fallback);
}
@@ -1890,7 +1911,7 @@
if (component1 != null && component2 != null) {
List<String> constraintPair = Arrays.asList(component1, component2);
List<List<String>> constraints = (List<List<String>>) computeIfAbsent(Config.TOPOLOGY_RAS_CONSTRAINTS,
- (k) -> new ArrayList<>(1));
+ (k) -> new ArrayList<>(1));
constraints.add(constraintPair);
}
}
@@ -1906,8 +1927,6 @@
/**
* Set the priority for a topology.
- *
- * @param priority
*/
public void setTopologyPriority(int priority) {
this.put(Config.TOPOLOGY_PRIORITY, priority);
@@ -1919,6 +1938,7 @@
private static final String HOSTNAME_PATTERN = "_HOST";
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static String getBlobstoreHDFSPrincipal(Map conf) throws UnknownHostException {
String principal = (String) conf.get(Config.BLOBSTORE_HDFS_PRINCIPAL);
if (principal != null) {
diff --git a/storm-client/src/jvm/org/apache/storm/ILocalCluster.java b/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
index 1c420de..3c223eb 100644
--- a/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
+++ b/storm-client/src/jvm/org/apache/storm/ILocalCluster.java
@@ -28,26 +28,24 @@
*/
public interface ILocalCluster extends AutoCloseable {
/**
- * Submit a topology to be run in local mode
+ * Submit a topology to be run in local mode.
*
* @param topologyName the name of the topology to use
* @param conf the config for the topology
* @param topology the topology itself.
* @return an AutoCloseable that will kill the topology.
- *
* @throws TException on any error from nimbus
*/
ILocalTopology submitTopology(String topologyName, Map<String, Object> conf, StormTopology topology) throws TException;
/**
- * Submit a topology to be run in local mode
+ * Submit a topology to be run in local mode.
*
* @param topologyName the name of the topology to use
* @param conf the config for the topology
* @param topology the topology itself.
* @param submitOpts options for topology
* @return an AutoCloseable that will kill the topology.
- *
* @throws TException on any error from nimbus
*/
ILocalTopology submitTopologyWithOpts(String topologyName, Map<String, Object> conf, StormTopology topology,
@@ -63,7 +61,7 @@
void uploadNewCredentials(String topologyName, Credentials creds) throws TException;
/**
- * Kill a topology (if you are not using ILocalTopology)
+ * Kill a topology (if you are not using ILocalTopology).
*
* @param topologyName the name of the topology
* @throws TException on any error from nimbus
@@ -71,7 +69,7 @@
void killTopology(String topologyName) throws TException;
/**
- * Kill a topology (if you are not using ILocalTopology)
+ * Kill a topology (if you are not using ILocalTopology).
*
* @param topologyName the name of the topology
* @param options for how to kill the topology
@@ -80,7 +78,7 @@
void killTopologyWithOpts(String name, KillOptions options) throws TException;
/**
- * Activate a topology
+ * Activate a topology.
*
* @param topologyName the name of the topology to activate
* @throws TException on any error from nimbus
@@ -88,7 +86,7 @@
void activate(String topologyName) throws TException;
/**
- * Deactivate a topology
+ * Deactivate a topology.
*
* @param topologyName the name of the topology to deactivate
* @throws TException on any error from nimbus
@@ -96,7 +94,7 @@
void deactivate(String topologyName) throws TException;
/**
- * Rebalance a topology
+ * Rebalance a topology.
*
* @param name the name of the topology
* @param options options for rebalanceing the topology.
@@ -113,38 +111,35 @@
void shutdown();
/**
- * The config of a topology as a JSON string
+ * The config of a topology as a JSON string.
*
* @param id the id of the topology (not the name)
* @return The config of a topology as a JSON string
- *
* @throws TException on any error from nimbus
*/
String getTopologyConf(String id) throws TException;
/**
- * Get the compiled storm topology
+ * Get the compiled storm topology.
*
* @param id the id of the topology (not the name)
* @return the compiled storm topology
- *
* @throws TException on any error from nimbus
*/
StormTopology getTopology(String id) throws TException;
/**
+ * Get cluster information.
* @return a summary of the current state of the cluster
- *
* @throws TException on any error from nimbus
*/
ClusterSummary getClusterInfo() throws TException;
/**
- * Get the state of a topology
+ * Get the state of a topology.
*
* @param id the id of the topology (not the name)
* @return the state of a topology
- *
* @throws TException on any error from nimbus
*/
TopologyInfo getTopologyInfo(String id) throws TException;
@@ -160,7 +155,6 @@
* Advance the cluster time when the cluster is using SimulatedTime. This is intended for internal testing only.
*
* @param secs the number of seconds to advance time
- * @throws InterruptedException
*/
void advanceClusterTime(int secs) throws InterruptedException;
@@ -169,7 +163,6 @@
*
* @param secs the number of seconds to advance time
* @param steps the number of steps we should take when advancing simulated time
- * @throws InterruptedException
*/
void advanceClusterTime(int secs, int step) throws InterruptedException;
diff --git a/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java b/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
index 1a8916d..037239b 100644
--- a/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
+++ b/storm-client/src/jvm/org/apache/storm/ILocalDRPC.java
@@ -16,17 +16,18 @@
import org.apache.storm.generated.DistributedRPC;
import org.apache.storm.generated.DistributedRPCInvocations;
-
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPCInvocations.Iface, Shutdownable, AutoCloseable {
/**
* Get the ID of the service. This is used internally if multiple local DRPC clusters are in use at one time.
*/
- public String getServiceId();
+ String getServiceId();
/**
+ * Shutdown.
* @deprecated use {@link #close()} instead
*/
@Deprecated
@Override
- public void shutdown();
+ void shutdown();
}
diff --git a/storm-client/src/jvm/org/apache/storm/StormSubmitter.java b/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
index 837aecd..ed97e13 100644
--- a/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
+++ b/storm-client/src/jvm/org/apache/storm/StormSubmitter.java
@@ -42,11 +42,11 @@
import org.apache.storm.security.auth.IAutoCredentials;
import org.apache.storm.shade.org.apache.commons.lang.StringUtils;
import org.apache.storm.shade.org.json.simple.JSONValue;
+import org.apache.storm.thrift.TException;
import org.apache.storm.utils.BufferFileInputStream;
import org.apache.storm.utils.NimbusClient;
import org.apache.storm.utils.Utils;
import org.apache.storm.validation.ConfigValidation;
-import org.apache.storm.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -63,6 +63,7 @@
return Utils.secureRandomLong() + ":" + Utils.secureRandomLong();
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static boolean validateZKDigestPayload(String payload) {
if (payload != null) {
Matcher m = zkDigestPattern.matcher(payload);
@@ -75,10 +76,9 @@
Map<String, Object> toRet = new HashMap<>();
String secretPayload = (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
// Is the topology ZooKeeper authentication configuration unset?
- if (!conf.containsKey(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) ||
- conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) == null ||
- !validateZKDigestPayload((String)
- conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD))) {
+ if (!conf.containsKey(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD)
+ || conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD) == null
+ || !validateZKDigestPayload((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD))) {
secretPayload = generateZookeeperDigestSecretPayload();
LOG.info("Generated ZooKeeper secret payload for MD5-digest: " + secretPayload);
}
@@ -187,17 +187,29 @@
}
/**
+ * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed.
+ *
+ * @param name the name of the storm.
+ * @param topoConf the topology-specific configuration. See {@link Config}.
+ * @param topology the processing to execute.
+ * @param opts to manipulate the starting of the topology
+ * @param progressListener to track the progress of the jar upload process
+ * @throws AlreadyAliveException if a topology with this name is already running
+ * @throws InvalidTopologyException if an invalid topology was submitted
+ * @throws AuthorizationException if authorization is failed
+ * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
+ */
+ @SuppressWarnings("unchecked")
+ public static void submitTopology(String name, Map<String, Object> topoConf, StormTopology topology, SubmitOptions opts,
+ ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException,
+ AuthorizationException {
+ submitTopologyAs(name, topoConf, topology, opts, progressListener, null);
+ }
+
+ /**
* Submits a topology to run on the cluster as a particular user. A topology runs forever or until explicitly killed.
*
- * @param name
- * @param topoConf
- * @param topology
- * @param opts
- * @param progressListener
- * @param asUser The user as which this topology should be submitted.
- * @throws AlreadyAliveException
- * @throws InvalidTopologyException
- * @throws AuthorizationException
+ * @param asUser The user as which this topology should be submitted.
* @throws IllegalArgumentException thrown if configs will yield an unschedulable topology. validateConfs validates confs
* @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
*/
@@ -337,10 +349,7 @@
}
/**
- * @param name
- * @param asUser
- * @param topoConf
- * @param topology
+ * Invoke submitter hook.
* @thorws SubmitterHookException This is thrown when any Exception occurs during initialization or invocation of registered {@link
* ISubmitterHook}
*/
@@ -368,26 +377,6 @@
}
/**
- * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed.
- *
- * @param name the name of the storm.
- * @param topoConf the topology-specific configuration. See {@link Config}.
- * @param topology the processing to execute.
- * @param opts to manipulate the starting of the topology
- * @param progressListener to track the progress of the jar upload process
- * @throws AlreadyAliveException if a topology with this name is already running
- * @throws InvalidTopologyException if an invalid topology was submitted
- * @throws AuthorizationException if authorization is failed
- * @thorws SubmitterHookException if any Exception occurs during initialization or invocation of registered {@link ISubmitterHook}
- */
- @SuppressWarnings("unchecked")
- public static void submitTopology(String name, Map<String, Object> topoConf, StormTopology topology, SubmitOptions opts,
- ProgressListener progressListener) throws AlreadyAliveException, InvalidTopologyException,
- AuthorizationException {
- submitTopologyAs(name, topoConf, topology, opts, progressListener, null);
- }
-
- /**
* Submits a topology to run on the cluster with a progress bar. A topology runs forever or until explicitly killed.
*
* @param name the name of the storm.
@@ -455,7 +444,7 @@
}
/**
- * Submit jar file
+ * Submit jar file.
*
* @param conf the topology-specific configuration. See {@link Config}.
* @param localJar file path of the jar file to submit
@@ -465,6 +454,18 @@
return submitJar(conf, localJar, null);
}
+ /**
+ * Submit jar file.
+ *
+ * @param conf the topology-specific configuration. See {@link Config}.
+ * @param localJar file path of the jar file to submit
+ * @param listener progress listener to track the jar file upload
+ * @return the remote location of the submitted jar
+ */
+ public static String submitJar(Map<String, Object> conf, String localJar, ProgressListener listener) {
+ return submitJarAs(conf, localJar, listener, (String) null);
+ }
+
public static String submitJarAs(Map<String, Object> conf, String localJar, ProgressListener listener, NimbusClient client) {
if (localJar == null) {
throw new RuntimeException(
@@ -518,18 +519,6 @@
}
}
- /**
- * Submit jar file
- *
- * @param conf the topology-specific configuration. See {@link Config}.
- * @param localJar file path of the jar file to submit
- * @param listener progress listener to track the jar file upload
- * @return the remote location of the submitted jar
- */
- public static String submitJar(Map<String, Object> conf, String localJar, ProgressListener listener) {
- return submitJarAs(conf, localJar, listener, (String) null);
- }
-
private static void validateConfs(Map<String, Object> topoConf, StormTopology topology) throws IllegalArgumentException,
InvalidTopologyException, AuthorizationException {
ConfigValidation.validateTopoConf(topoConf);
@@ -537,11 +526,11 @@
}
/**
- * Interface use to track progress of file upload
+ * Interface use to track progress of file upload.
*/
public interface ProgressListener {
/**
- * called before file is uploaded
+ * called before file is uploaded.
*
* @param srcFile - jar file to be uploaded
* @param targetFile - destination file
@@ -550,7 +539,7 @@
public void onStart(String srcFile, String targetFile, long totalBytes);
/**
- * called whenever a chunk of bytes is uploaded
+ * called whenever a chunk of bytes is uploaded.
*
* @param srcFile - jar file to be uploaded
* @param targetFile - destination file
@@ -560,7 +549,7 @@
public void onProgress(String srcFile, String targetFile, long bytesUploaded, long totalBytes);
/**
- * called when the file is uploaded
+ * called when the file is uploaded.
*
* @param srcFile - jar file to be uploaded
* @param targetFile - destination file
diff --git a/storm-client/src/jvm/org/apache/storm/StormTimer.java b/storm-client/src/jvm/org/apache/storm/StormTimer.java
index bcd39d3..a68384b 100644
--- a/storm-client/src/jvm/org/apache/storm/StormTimer.java
+++ b/storm-client/src/jvm/org/apache/storm/StormTimer.java
@@ -65,6 +65,10 @@
scheduleMs(Time.secsToMillisLong(delaySecs), func, checkActive, jitterMs);
}
+ public void schedule(int delaySecs, Runnable func) {
+ schedule(delaySecs, func, true, 0);
+ }
+
/**
* Same as schedule with millisecond resolution.
*
@@ -88,10 +92,6 @@
task.add(new QueueEntry(endTimeMs, func, id));
}
- public void schedule(int delaySecs, Runnable func) {
- schedule(delaySecs, func, true, 0);
- }
-
public void scheduleMs(long delayMs, Runnable func) {
scheduleMs(delayMs, func, true, 0);
}
diff --git a/storm-client/src/jvm/org/apache/storm/Thrift.java b/storm-client/src/jvm/org/apache/storm/Thrift.java
index 1745cd1..acba4ae 100644
--- a/storm-client/src/jvm/org/apache/storm/Thrift.java
+++ b/storm-client/src/jvm/org/apache/storm/Thrift.java
@@ -218,8 +218,8 @@
}
public static SpoutSpec prepareSerializedSpoutDetails(IRichSpout spout, Map<String, StreamInfo> outputs) {
- return new SpoutSpec(ComponentObject.serialized_java
- (Utils.javaSerialize(spout)), prepareComponentCommon(new HashMap<>(), outputs, null, null));
+ return new SpoutSpec(ComponentObject.serialized_java(Utils.javaSerialize(spout)),
+ prepareComponentCommon(new HashMap<>(), outputs, null, null));
}
public static Bolt prepareSerializedBoltDetails(Map<GlobalStreamId, Grouping> inputs, IBolt bolt, Map<String, StreamInfo> outputs,
@@ -256,17 +256,17 @@
return details;
}
- public static StormTopology buildTopology(HashMap<String, SpoutDetails> spoutMap,
- HashMap<String, BoltDetails> boltMap, HashMap<String, StateSpoutSpec> stateMap) {
- return buildTopology(spoutMap, boltMap);
- }
-
private static void addInputs(BoltDeclarer declarer, Map<GlobalStreamId, Grouping> inputs) {
for (Entry<GlobalStreamId, Grouping> entry : inputs.entrySet()) {
declarer.grouping(entry.getKey(), entry.getValue());
}
}
+ public static StormTopology buildTopology(HashMap<String, SpoutDetails> spoutMap,
+ HashMap<String, BoltDetails> boltMap, HashMap<String, StateSpoutSpec> stateMap) {
+ return buildTopology(spoutMap, boltMap);
+ }
+
public static StormTopology buildTopology(Map<String, SpoutDetails> spoutMap, Map<String, BoltDetails> boltMap) {
TopologyBuilder builder = new TopologyBuilder();
for (Entry<String, SpoutDetails> entry : spoutMap.entrySet()) {
diff --git a/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java b/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
index 1cd0de9..0a57ac1 100644
--- a/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
+++ b/storm-client/src/jvm/org/apache/storm/annotation/InterfaceStability.java
@@ -18,7 +18,6 @@
/**
* Annotation to inform users of how much to rely on a particular package, class or method not changing over time.
- * </ul>
*/
@InterfaceStability.Evolving
public class InterfaceStability {
diff --git a/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java b/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java
index 03a400c..8854e38 100644
--- a/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java
+++ b/storm-client/src/jvm/org/apache/storm/assignments/InMemoryAssignmentBackend.java
@@ -24,6 +24,7 @@
/**
* An assignment backend which will keep all assignments and id-info in memory. Only used if no backend is specified internal.
+ *
* <p>About thread safe: idToAssignment,idToName,nameToId are all memory cache in nimbus local, for
* <ul>
* <li>idToAssignment: nimbus will modify it and supervisors will sync it at fixed interval,
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
index 6cf9df9..c9219c5 100644
--- a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStore.java
@@ -37,16 +37,17 @@
import org.slf4j.LoggerFactory;
/**
- * Provides a way to store blobs that can be downloaded. Blobs must be able to be uploaded and listed from Nimbus, and downloaded from the
- * Supervisors. It is a key value based store. Key being a string and value being the blob data.
+ * Provides a way to store blobs that can be downloaded. Blobs must be able to be uploaded and listed from Nimbus, and
+ * downloaded from the Supervisors. It is a key value based store. Key being a string and value being the blob data.
*
- * ACL checking must take place against the provided subject. If the blob store does not support Security it must validate that all ACLs set
- * are always WORLD, everything.
+ * <p>ACL checking must take place against the provided subject. If the blob store does not support Security it must
+ * validate that all ACLs set are always WORLD, everything.
*
- * The users can upload their blobs through the blob store command line. The command line also allows us to update and delete blobs.
+ * <p>The users can upload their blobs through the blob store command line. The command line also allows us to update
+ * and delete blobs.
*
- * Modifying the replication factor only works for HdfsBlobStore as for the LocalFsBlobStore the replication is dependent on the number of
- * Nimbodes available.
+ * <p>Modifying the replication factor only works for HdfsBlobStore as for the LocalFsBlobStore the replication is
+ * dependent on the number of Nimbodes available.
*/
public abstract class BlobStore implements Shutdownable, AutoCloseable {
protected static final String BASE_BLOBS_DIR_NAME = "blobs";
@@ -54,9 +55,9 @@
private static final KeyFilter<String> TO_TOPO_ID = (key) -> ConfigUtils.getIdFromBlobKey(key);
/**
- * Validates key checking for potentially harmful patterns
+ * Validates key checking for potentially harmful patterns.
*
- * @param key Key for the blob.
+ * @param key Key for the blob
*/
public static final void validateKey(String key) throws IllegalArgumentException {
if (!Utils.isValidKey(key)) {
@@ -65,11 +66,11 @@
}
/**
- * Allows us to initialize the blob store
+ * Allows us to initialize the blob store.
*
* @param conf The storm configuration
* @param baseDir The directory path to store the blobs
- * @param nimbusInfo Contains the nimbus host, port and leadership information.
+ * @param nimbusInfo Contains the nimbus host, port and leadership information
*/
public abstract void prepare(Map<String, Object> conf, String baseDir, NimbusInfo nimbusInfo, ILeaderElector leaderElector);
@@ -85,109 +86,163 @@
/**
* Creates the blob.
*
- * @param key Key for the blob.
+ * @param key Key for the blob
* @param meta Metadata which contains the acls information
- * @param who Is the subject creating the blob.
- * @return AtomicOutputStream returns a stream into which the data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyAlreadyExistsException
+ * @param who Is the subject creating the blob
+ * @return AtomicOutputStream returns a stream into which the data can be written
*/
public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException,
KeyAlreadyExistsException;
/**
+ * Wrapper called to create the blob which contains the byte data.
+ *
+ * @param key Key for the blob
+ * @param data Byte data that needs to be uploaded
+ * @param meta Metadata which contains the acls information
+ * @param who Is the subject creating the blob
+ */
+ public void createBlob(String key, byte[] data, SettableBlobMeta meta, Subject who) throws AuthorizationException,
+ KeyAlreadyExistsException, IOException {
+ AtomicOutputStream out = null;
+ try {
+ out = createBlob(key, meta, who);
+ out.write(data);
+ out.close();
+ out = null;
+ } finally {
+ if (out != null) {
+ out.cancel();
+ }
+ }
+ }
+
+ /**
+ * Wrapper called to create the blob which contains the byte data.
+ *
+ * @param key Key for the blob
+ * @param in InputStream from which the data is read to be written as a part of the blob
+ * @param meta Metadata which contains the acls information
+ * @param who Is the subject creating the blob
+ */
+ public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException,
+ KeyAlreadyExistsException, IOException {
+ AtomicOutputStream out = null;
+ try {
+ out = createBlob(key, meta, who);
+ byte[] buffer = new byte[2048];
+ int len = 0;
+ while ((len = in.read(buffer)) > 0) {
+ out.write(buffer, 0, len);
+ }
+ out.close();
+ out = null;
+ } finally {
+ try {
+ if (out != null) {
+ out.cancel();
+ }
+ in.close();
+ } catch (IOException throwaway) {
+ // Ignored
+ }
+ }
+ }
+
+ /**
* Updates the blob data.
*
- * @param key Key for the blob.
- * @param who Is the subject having the write privilege for the blob.
- * @return AtomicOutputStream returns a stream into which the data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key Key for the blob
+ * @param who Is the subject having the write privilege for the blob
+ * @return AtomicOutputStream returns a stream into which the data can be written
*/
public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
/**
+ * Wrapper called to create the blob which contains the byte data.
+ *
+ * @param key Key for the blob
+ * @param data Byte data that needs to be uploaded
+ * @param who Is the subject creating the blob
+ */
+ public void updateBlob(String key, byte[] data, Subject who) throws AuthorizationException, IOException, KeyNotFoundException {
+ AtomicOutputStream out = null;
+ try {
+ out = updateBlob(key, who);
+ out.write(data);
+ out.close();
+ out = null;
+ } finally {
+ if (out != null) {
+ out.cancel();
+ }
+ }
+ }
+
+ /**
* Gets the current version of metadata for a blob to be viewed by the user or downloaded by the supervisor.
*
- * @param key Key for the blob.
- * @param who Is the subject having the read privilege for the blob.
- * @return AtomicOutputStream returns a stream into which the data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key Key for the blob
+ * @param who Is the subject having the read privilege for the blob
+ * @return AtomicOutputStream returns a stream into which the data can be written
*/
public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
/**
- * Sets leader elector (only used by LocalFsBlobStore to help sync blobs between Nimbi
- * @param leaderElector
+ * Sets leader elector (only used by LocalFsBlobStore to help sync blobs between Nimbi.
*/
public abstract void setLeaderElector(ILeaderElector leaderElector);
+
/**
* Sets the metadata with renewed acls for the blob.
*
- * @param key Key for the blob.
- * @param meta Metadata which contains the updated acls information.
- * @param who Is the subject having the write privilege for the blob.
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key Key for the blob
+ * @param meta Metadata which contains the updated acls information
+ * @param who Is the subject having the write privilege for the blob
*/
public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
/**
* Deletes the blob data and metadata.
*
- * @param key Key for the blob.
- * @param who Is the subject having write privilege for the blob.
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key Key for the blob
+ * @param who Is the subject having write privilege for the blob
*/
public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
/**
- * Gets the InputStream to read the blob details
+ * Gets the InputStream to read the blob details.
*
- * @param key Key for the blob.
- * @param who Is the subject having the read privilege for the blob.
- * @return InputStreamWithMeta has the additional file length and version information.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key Key for the blob
+ * @param who Is the subject having the read privilege for the blob
+ * @return InputStreamWithMeta has the additional file length and version information
*/
public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
/**
* Returns an iterator with all the list of keys currently available on the blob store.
*
- * @return Iterator<String>
+ * @return {@code Iterator<String>}
*/
public abstract Iterator<String> listKeys();
/**
* Gets the replication factor of the blob.
*
- * @param key Key for the blob.
- * @param who Is the subject having the read privilege for the blob.
- * @return BlobReplication object containing the replication factor for the blob.
- *
- * @throws Exception
+ * @param key Key for the blob
+ * @param who Is the subject having the read privilege for the blob
+ * @return BlobReplication object containing the replication factor for the blob
*/
public abstract int getBlobReplication(String key, Subject who) throws Exception;
/**
* Modifies the replication factor of the blob.
*
- * @param key Key for the blob.
- * @param replication The replication factor the blob has to be set.
+ * @param key Key for the blob
+ * @param replication The replication factor the blob has to be set
* @param who Is the subject having the update privilege for the blob
- * @return BlobReplication object containing the updated replication factor for the blob.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
- * @throws IOException
+ * @return BlobReplication object containing the updated replication factor for the blob
*/
public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException,
IOException;
@@ -218,99 +273,11 @@
}
/**
- * Wrapper called to create the blob which contains the byte data
- *
- * @param key Key for the blob.
- * @param data Byte data that needs to be uploaded.
- * @param meta Metadata which contains the acls information
- * @param who Is the subject creating the blob.
- * @throws AuthorizationException
- * @throws KeyAlreadyExistsException
- * @throws IOException
- */
- public void createBlob(String key, byte[] data, SettableBlobMeta meta, Subject who) throws AuthorizationException,
- KeyAlreadyExistsException, IOException {
- AtomicOutputStream out = null;
- try {
- out = createBlob(key, meta, who);
- out.write(data);
- out.close();
- out = null;
- } finally {
- if (out != null) {
- out.cancel();
- }
- }
- }
-
- /**
- * Wrapper called to create the blob which contains the byte data
- *
- * @param key Key for the blob.
- * @param data Byte data that needs to be uploaded.
- * @param who Is the subject creating the blob.
- * @throws AuthorizationException
- * @throws IOException
- * @throws KeyNotFoundException
- */
- public void updateBlob(String key, byte[] data, Subject who) throws AuthorizationException, IOException, KeyNotFoundException {
- AtomicOutputStream out = null;
- try {
- out = updateBlob(key, who);
- out.write(data);
- out.close();
- out = null;
- } finally {
- if (out != null) {
- out.cancel();
- }
- }
- }
-
- /**
- * Wrapper called to create the blob which contains the byte data
- *
- * @param key Key for the blob.
- * @param in InputStream from which the data is read to be written as a part of the blob.
- * @param meta Metadata which contains the acls information
- * @param who Is the subject creating the blob.
- * @throws AuthorizationException
- * @throws KeyAlreadyExistsException
- * @throws IOException
- */
- public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException,
- KeyAlreadyExistsException, IOException {
- AtomicOutputStream out = null;
- try {
- out = createBlob(key, meta, who);
- byte[] buffer = new byte[2048];
- int len = 0;
- while ((len = in.read(buffer)) > 0) {
- out.write(buffer, 0, len);
- }
- out.close();
- out = null;
- } finally {
- try {
- if (out != null) {
- out.cancel();
- }
- in.close();
- } catch (IOException throwaway) {
- // Ignored
- }
- }
- }
-
- /**
* Reads the blob from the blob store and writes it into the output stream.
*
- * @param key Key for the blob.
+ * @param key Key for the blob
* @param out Output stream
- * @param who Is the subject having read privilege for the blob.
- * @throws IOException
- * @throws KeyNotFoundException
- * @throws AuthorizationException
+ * @param who Is the subject having read privilege for the blob
*/
public void readBlobTo(String key, OutputStream out, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
InputStreamWithMeta in = getBlob(key, who);
@@ -332,13 +299,8 @@
/**
* Wrapper around readBlobTo which returns a ByteArray output stream.
*
- * @param key Key for the blob.
- * @param who Is the subject having the read privilege for the blob.
- * @return ByteArrayOutputStream
- *
- * @throws IOException
- * @throws KeyNotFoundException
- * @throws AuthorizationException
+ * @param key Key for the blob
+ * @param who Is the subject having the read privilege for the blob
*/
public byte[] readBlob(String key, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
@@ -349,6 +311,7 @@
}
/**
+ * Get IDs stored in blob store.
* @return a set of all of the topology ids with special data stored in the blob store.
*/
public Set<String> storedTopoIds() {
@@ -356,7 +319,7 @@
}
/**
- * Blob store implements its own version of iterator to list the blobs
+ * Blob store implements its own version of iterator to list the blobs.
*/
public static class KeyTranslationIterator implements Iterator<String> {
private Iterator<String> it = null;
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
index 3b59022..2270142 100644
--- a/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/BlobStoreAclHandler.java
@@ -46,7 +46,7 @@
public static final List<AccessControl> WORLD_EVERYTHING =
Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
- private final IPrincipalToLocal _ptol;
+ private final IPrincipalToLocal ptol;
private final IGroupMappingServiceProvider groupMappingServiceProvider;
private Set<String> supervisors;
private Set<String> admins;
@@ -54,7 +54,7 @@
private boolean doAclValidation;
public BlobStoreAclHandler(Map<String, Object> conf) {
- _ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
+ ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
if (conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN) != null) {
groupMappingServiceProvider = ClientAuthUtils.getGroupMappingServiceProviderPlugin(conf);
} else {
@@ -77,7 +77,7 @@
}
}
- private static AccessControlType parseACLType(String type) {
+ private static AccessControlType parseAclType(String type) {
if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
return AccessControlType.OTHER;
} else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
@@ -125,7 +125,7 @@
access = parts[2];
}
AccessControl ret = new AccessControl();
- ret.set_type(parseACLType(type));
+ ret.set_type(parseAclType(type));
ret.set_name(name);
ret.set_access(parseAccess(access));
return ret;
@@ -160,6 +160,7 @@
return ret.toString();
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
Set<String> aclUsers = new HashSet<>();
List<String> duplicateUsers = new ArrayList<>();
@@ -181,7 +182,7 @@
Set<String> user = new HashSet<String>();
if (who != null) {
for (Principal p : who.getPrincipals()) {
- user.add(_ptol.toLocal(p));
+ user.add(ptol.toLocal(p));
}
}
return user;
@@ -264,7 +265,6 @@
* 5 implies READ and ADMIN privileges.
* @param who Is the user against whom the permissions are validated for a key using the ACL and the mask.
* @param key Key used to identify the blob.
- * @throws AuthorizationException
*/
public void hasAnyPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
if (!doAclValidation) {
@@ -294,7 +294,6 @@
* 5 implies READ and ADMIN privileges.
* @param who Is the user against whom the permissions are validated for a key using the ACL and the mask.
* @param key Key used to identify the blob.
- * @throws AuthorizationException
*/
public void hasPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
if (!doAclValidation) {
@@ -318,7 +317,7 @@
}
public void normalizeSettableBlobMeta(String key, SettableBlobMeta meta, Subject who, int opMask) {
- meta.set_acl(normalizeSettableACLs(key, meta.get_acl(), who, opMask));
+ meta.set_acl(normalizeSettableAcls(key, meta.get_acl(), who, opMask));
}
private String namedPerms(int mask) {
@@ -351,7 +350,7 @@
}
}
- private List<AccessControl> removeBadACLs(List<AccessControl> accessControls) {
+ private List<AccessControl> removeBadAcls(List<AccessControl> accessControls) {
List<AccessControl> resultAcl = new ArrayList<AccessControl>();
for (AccessControl control : accessControls) {
if (control.get_type().equals(AccessControlType.OTHER) && (control.get_access() == 0)) {
@@ -364,12 +363,12 @@
return resultAcl;
}
- private final List<AccessControl> normalizeSettableACLs(String key, List<AccessControl> acls, Subject who,
+ private final List<AccessControl> normalizeSettableAcls(String key, List<AccessControl> acls, Subject who,
int opMask) {
- List<AccessControl> cleanAcls = removeBadACLs(acls);
+ List<AccessControl> cleanAcls = removeBadAcls(acls);
Set<String> userNames = getUserNamesFromSubject(who);
for (String user : userNames) {
- fixACLsForUser(cleanAcls, user, opMask);
+ fixAclsForUser(cleanAcls, user, opMask);
}
fixEmptyNameACLForUsers(cleanAcls, userNames, opMask);
if ((who == null || userNames.isEmpty()) && !worldEverything(acls)) {
@@ -393,39 +392,40 @@
return isWorldEverything;
}
- private void fixACLsForUser(List<AccessControl> acls, String user, int mask) {
- boolean foundUserACL = false;
- List<AccessControl> emptyUserACLs = new ArrayList<>();
+ private void fixAclsForUser(List<AccessControl> acls, String user, int mask) {
+ boolean foundUserAcl = false;
+ List<AccessControl> emptyUserAcls = new ArrayList<>();
for (AccessControl control : acls) {
if (control.get_type() == AccessControlType.USER) {
if (!control.is_set_name()) {
- emptyUserACLs.add(control);
+ emptyUserAcls.add(control);
} else if (control.get_name().equals(user)) {
int currentAccess = control.get_access();
if ((currentAccess & mask) != mask) {
control.set_access(currentAccess | mask);
}
- foundUserACL = true;
+ foundUserAcl = true;
}
}
}
// if ACLs have two user ACLs for empty user and principal, discard empty user ACL
- if (!emptyUserACLs.isEmpty() && foundUserACL) {
- acls.removeAll(emptyUserACLs);
+ if (!emptyUserAcls.isEmpty() && foundUserAcl) {
+ acls.removeAll(emptyUserAcls);
}
// add default user ACL when only empty user ACL is not present
- if (emptyUserACLs.isEmpty() && !foundUserACL) {
- AccessControl userACL = new AccessControl();
- userACL.set_type(AccessControlType.USER);
- userACL.set_name(user);
- userACL.set_access(mask);
- acls.add(userACL);
+ if (emptyUserAcls.isEmpty() && !foundUserAcl) {
+ AccessControl userAcl = new AccessControl();
+ userAcl.set_type(AccessControlType.USER);
+ userAcl.set_name(user);
+ userAcl.set_access(mask);
+ acls.add(userAcl);
}
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private void fixEmptyNameACLForUsers(List<AccessControl> acls, Set<String> users, int mask) {
List<AccessControl> aclsToAdd = new ArrayList<>();
List<AccessControl> aclsToRemove = new ArrayList<>();
@@ -455,7 +455,7 @@
Set<String> user = new HashSet<String>();
if (who != null) {
for (Principal p : who.getPrincipals()) {
- user.add(_ptol.toLocal(p));
+ user.add(ptol.toLocal(p));
}
}
return user;
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
index ee4387f..a83146d 100644
--- a/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/ClientBlobStore.java
@@ -25,16 +25,17 @@
import org.apache.storm.utils.Utils;
/**
- * The ClientBlobStore has two concrete implementations 1. NimbusBlobStore 2. HdfsClientBlobStore
+ * The ClientBlobStore has two concrete implementations 1. NimbusBlobStore 2. HdfsClientBlobStore.
*
- * Create, update, read and delete are some of the basic operations defined by this interface. Each operation is validated for permissions
- * against an user. We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS configuration. NIMBUS_ADMINS are given READ, WRITE and ADMIN
- * access whereas the SUPERVISOR_ADMINS are given READ access in order to read and download the blobs form the nimbus.
+ * <p>Create, update, read and delete are some of the basic operations defined by this interface. Each operation is
+ * validated for permissions against an user. We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS configuration.
+ * NIMBUS_ADMINS are given READ, WRITE and ADMIN access whereas the SUPERVISOR_ADMINS are given READ access in order to
+ * read and download the blobs form the nimbus.
*
- * The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER who has read, write or
- * admin privileges in order to perform respective operations on the blob.
+ * <p>The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER
+ * who has read, write or admin privileges in order to perform respective operations on the blob.
*
- * For more detailed implementation
+ * <p>For more detailed implementation
*
* @see org.apache.storm.blobstore.NimbusBlobStore
*/
@@ -50,19 +51,16 @@
/**
* Sets up the client API by parsing the configs.
*
- * @param conf The storm conf containing the config details.
+ * @param conf The storm conf containing the config details
*/
public abstract void prepare(Map<String, Object> conf);
/**
* Client facing API to create a blob.
*
- * @param key blob key name.
- * @param meta contains ACL information.
- * @return AtomicOutputStream returns an output stream into which data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyAlreadyExistsException
+ * @param key blob key name
+ * @param meta contains ACL information
+ * @return AtomicOutputStream returns an output stream into which data can be written
*/
protected abstract AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException,
KeyAlreadyExistsException;
@@ -70,22 +68,16 @@
/**
* Client facing API to update a blob.
*
- * @param key blob key name.
- * @return AtomicOutputStream returns an output stream into which data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @return AtomicOutputStream returns an output stream into which data can be written
*/
public abstract AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException;
/**
* Client facing API to read the metadata information.
*
- * @param key blob key name.
- * @return AtomicOutputStream returns an output stream into which data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @return AtomicOutputStream returns an output stream into which data can be written
*/
public abstract ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException;
@@ -98,34 +90,28 @@
/**
* Client facing API to set the metadata for a blob.
*
- * @param key blob key name.
- * @param meta contains ACL information.
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @param meta contains ACL information
*/
protected abstract void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException;
/**
* Client facing API to delete a blob.
*
- * @param key blob key name.
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
*/
public abstract void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException;
/**
* Client facing API to read a blob.
*
- * @param key blob key name.
- * @return an InputStream to read the metadata for a blob.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @return an InputStream to read the metadata for a blob
*/
public abstract InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException;
/**
+ * List keys.
* @return Iterator for a list of keys currently present in the blob store.
*/
public abstract Iterator<String> listKeys();
@@ -133,23 +119,17 @@
/**
* Client facing API to read the replication of a blob.
*
- * @param key blob key name.
- * @return int indicates the replication factor of a blob.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @return int indicates the replication factor of a blob
*/
public abstract int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException;
/**
* Client facing API to update the replication of a blob.
*
- * @param key blob key name.
- * @param replication int indicates the replication factor a blob has to be set.
- * @return int indicates the replication factor of a blob.
- *
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @param replication int indicates the replication factor a blob has to be set
+ * @return int indicates the replication factor of a blob
*/
public abstract int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException;
@@ -163,10 +143,8 @@
public abstract boolean setClient(Map<String, Object> conf, NimbusClient client);
/**
- * Creates state inside a zookeeper. Required for blobstore to write to zookeeper when Nimbus HA is turned on in order to maintain state
- * consistency
- *
- * @param key
+ * Creates state inside a zookeeper. Required for blobstore to write to zookeeper when Nimbus HA is turned on in
+ * order to maintain state consistency.
*/
public abstract void createStateInZookeeper(String key);
@@ -176,12 +154,9 @@
/**
* Client facing API to create a blob.
*
- * @param key blob key name.
- * @param meta contains ACL information.
- * @return AtomicOutputStream returns an output stream into which data can be written.
- *
- * @throws AuthorizationException
- * @throws KeyAlreadyExistsException
+ * @param key blob key name
+ * @param meta contains ACL information
+ * @return AtomicOutputStream returns an output stream into which data can be written
*/
public final AtomicOutputStream createBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
if (meta != null && meta.is_set_acl()) {
@@ -193,10 +168,8 @@
/**
* Client facing API to set the metadata for a blob.
*
- * @param key blob key name.
- * @param meta contains ACL information.
- * @throws AuthorizationException
- * @throws KeyNotFoundException
+ * @param key blob key name
+ * @param meta contains ACL information
*/
public final void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException {
if (meta != null && meta.is_set_acl()) {
diff --git a/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java b/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
index 8b8a549..9a8f034 100644
--- a/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
+++ b/storm-client/src/jvm/org/apache/storm/blobstore/NimbusBlobStore.java
@@ -35,7 +35,7 @@
* NimbusBlobStore is a USER facing client API to perform basic operations such as create, update, delete and read for local and hdfs blob
* store.
*
- * For local blob store it is also the client facing API for supervisor in order to download blobs from nimbus.
+ * <p>For local blob store it is also the client facing API for supervisor in order to download blobs from nimbus.
*/
public class NimbusBlobStore extends ClientBlobStore implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
@@ -199,6 +199,7 @@
}
@Override
+ @SuppressWarnings("checkstyle:NoFinalizer")
protected void finalize() {
shutdown();
}
@@ -325,6 +326,11 @@
}
}
+ @Override
+ public synchronized int read(byte[] b) throws IOException {
+ return read(b, 0, b.length);
+ }
+
private boolean isEmpty() {
return buffer == null || offset >= end;
}
@@ -346,11 +352,6 @@
}
@Override
- public synchronized int read(byte[] b) throws IOException {
- return read(b, 0, b.length);
- }
-
- @Override
public synchronized int available() {
return buffer == null ? 0 : (end - offset);
}
diff --git a/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java b/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
index e556a63..6246966 100644
--- a/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/bolt/JoinBolt.java
@@ -12,7 +12,6 @@
package org.apache.storm.bolt;
-
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
@@ -29,7 +28,6 @@
import org.apache.storm.windowing.TimestampExtractor;
import org.apache.storm.windowing.TupleWindow;
-
public class JoinBolt extends BaseWindowedBolt {
protected final Selector selectorType;
@@ -75,11 +73,15 @@
}
/**
- * Performs inner Join with the newStream. SQL : from priorStream inner join newStream on newStream.field = priorStream.field1 same
- * as: new WindowedQueryBolt(priorStream,field1). join(newStream, field, priorStream);
- *
- * Note: priorStream must be previously joined. Valid ex: new WindowedQueryBolt(s1,k1). join(s2,k2, s1). join(s3,k3, s2); Invalid ex:
- * new WindowedQueryBolt(s1,k1). join(s3,k3, s2). join(s2,k2, s1);
+ * Performs inner Join with the newStream.
+ * SQL:
+ * <code>from priorStream inner join newStream on newStream.field = priorStream.field1</code>
+ * same as:
+ * <code>new WindowedQueryBolt(priorStream,field1). join(newStream, field, priorStream);</code>
+ * Note: priorStream must be previously joined. Valid ex:
+ * <code>new WindowedQueryBolt(s1,k1). join(s2,k2, s1). join(s3,k3, s2);</code>
+ * Invalid ex:
+ * <code>new WindowedQueryBolt(s1,k1). join(s3,k3, s2). join(s2,k2, s1);</code>
*
* @param newStream Either stream name or name of upstream component
* @param field the field on which to perform the join
@@ -92,7 +94,7 @@
* Performs left Join with the newStream. SQL : from stream1 left join stream2 on stream2.field = stream1.field1 same as: new
* WindowedQueryBolt(stream1, field1). leftJoin(stream2, field, stream1);
*
- * Note: priorStream must be previously joined Valid ex: new WindowedQueryBolt(s1,k1). leftJoin(s2,k2, s1). leftJoin(s3,k3, s2);
+ * <p>Note: priorStream must be previously joined Valid ex: new WindowedQueryBolt(s1,k1). leftJoin(s2,k2, s1). leftJoin(s3,k3, s2);
* Invalid ex: new WindowedQueryBolt(s1,k1). leftJoin(s3,k3, s2). leftJoin(s2,k2, s1);
*
* @param newStream Either a name of a stream or an upstream component
@@ -122,9 +124,6 @@
* Key names are supported for nested types: e.g: .select("outerKey1.innerKey1, outerKey1.innerKey2, stream3:outerKey2.innerKey3)" Inner
* types (non leaf) must be Map<> in order to support nested lookup using this dot notation This selected fields implicitly declare the
* output fieldNames for the bolt based.
- *
- * @param commaSeparatedKeys
- * @return
*/
public JoinBolt select(String commaSeparatedKeys) {
String[] fieldNames = commaSeparatedKeys.split(",");
@@ -339,8 +338,8 @@
protected Object lookupField(FieldSelector fieldSelector, Tuple tuple) {
// very stream name matches, it stream name was specified
- if (fieldSelector.streamName != null &&
- !fieldSelector.streamName.equalsIgnoreCase(getStreamSelector(tuple))) {
+ if (fieldSelector.streamName != null
+ && !fieldSelector.streamName.equalsIgnoreCase(getStreamSelector(tuple))) {
return null;
}
@@ -435,13 +434,18 @@
STREAM, SOURCE
}
- protected enum JoinType {INNER, LEFT, RIGHT, OUTER}
+ protected enum JoinType {
+ INNER,
+ LEFT,
+ RIGHT,
+ OUTER
+ }
/**
- * Describes how to join the other stream with the current stream
+ * Describes how to join the other stream with the current stream.
*/
protected static class JoinInfo implements Serializable {
- final static long serialVersionUID = 1L;
+ static final long serialVersionUID = 1L;
private JoinType joinType; // nature of join
private FieldSelector field; // field for the current stream
@@ -505,6 +509,8 @@
}
/**
+ * Constructor.
+ *
* @param stream name of stream
* @param fieldDescriptor Simple fieldDescriptor like "x.y.z" and w/o a 'stream1:' stream qualifier.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java b/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
index 08672fa..30f9b5d 100644
--- a/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
+++ b/storm-client/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
@@ -14,6 +14,7 @@
import org.apache.storm.shade.org.apache.zookeeper.Watcher;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface ZKStateChangedCallback {
public void changed(Watcher.Event.EventType type, String path);
}
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 145ec99..c0bacc4 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -224,7 +224,7 @@
}
/**
- * Get the path to secret keys for a specific topology
+ * Get the path to secret keys for a specific topology.
*
* @param type the service the secret is for.
* @param topologyId the topology the secret is for.
@@ -254,11 +254,7 @@
}
/**
- * Ensures that we only return heartbeats for executors assigned to this worker
- *
- * @param executors
- * @param workerHeartbeat
- * @return
+ * Ensures that we only return heartbeats for executors assigned to this worker.
*/
public static Map<ExecutorInfo, ExecutorBeat> convertExecutorBeats(List<ExecutorInfo> executors,
ClusterWorkerHeartbeat workerHeartbeat) {
@@ -276,18 +272,18 @@
return executorWhb;
}
- public static IStateStorage mkStateStorage(Map<String, Object> config, Map<String, Object> auth_conf,
+ public static IStateStorage mkStateStorage(Map<String, Object> config, Map<String, Object> authConf,
ClusterStateContext context) throws Exception {
- return _instance.mkStateStorageImpl(config, auth_conf, context);
+ return _instance.mkStateStorageImpl(config, authConf, context);
}
- public static IStormClusterState mkStormClusterState(Object StateStorage, ILocalAssignmentsBackend backend,
+ public static IStormClusterState mkStormClusterState(Object stateStorage, ILocalAssignmentsBackend backend,
ClusterStateContext context) throws Exception {
- return _instance.mkStormClusterStateImpl(StateStorage, backend, context);
+ return _instance.mkStormClusterStateImpl(stateStorage, backend, context);
}
- public static IStormClusterState mkStormClusterState(Object StateStorage, ClusterStateContext context) throws Exception {
- return _instance.mkStormClusterStateImpl(StateStorage, LocalAssignmentsBackendFactory.getDefault(), context);
+ public static IStormClusterState mkStormClusterState(Object stateStorage, ClusterStateContext context) throws Exception {
+ return _instance.mkStormClusterStateImpl(stateStorage, LocalAssignmentsBackendFactory.getDefault(), context);
}
public static String stringifyError(Throwable error) {
@@ -302,13 +298,13 @@
if (stateStorage instanceof IStateStorage) {
return new StormClusterStateImpl((IStateStorage) stateStorage, backend, context, false);
} else {
- IStateStorage Storage = _instance.mkStateStorageImpl((Map<String, Object>) stateStorage,
+ IStateStorage storage = _instance.mkStateStorageImpl((Map<String, Object>) stateStorage,
(Map<String, Object>) stateStorage, context);
- return new StormClusterStateImpl(Storage, backend, context, true);
+ return new StormClusterStateImpl(storage, backend, context, true);
}
}
- public IStateStorage mkStateStorageImpl(Map<String, Object> config, Map<String, Object> auth_conf, ClusterStateContext context) throws
+ public IStateStorage mkStateStorageImpl(Map<String, Object> config, Map<String, Object> authConf, ClusterStateContext context) throws
Exception {
String className = null;
IStateStorage stateStorage = null;
@@ -319,7 +315,7 @@
}
Class clazz = Class.forName(className);
StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
- stateStorage = storageFactory.mkStore(config, auth_conf, context);
+ stateStorage = storageFactory.mkStore(config, authConf, context);
return stateStorage;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
index b673932..da16ea3 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -19,14 +19,16 @@
import org.apache.storm.shade.org.apache.zookeeper.data.ACL;
/**
- * StateStorage provides the API for the pluggable state store used by the Storm daemons. Data is stored in path/value format, and the store
- * supports listing sub-paths at a given path. All data should be available across all nodes with eventual consistency.
+ * StateStorage provides the API for the pluggable state store used by the Storm daemons. Data is stored in path/value
+ * format, and the store supports listing sub-paths at a given path. All data should be available across all nodes with
+ * eventual consistency.
*
- * IMPORTANT NOTE: Heartbeats have different api calls used to interact with them. The root path (/) may or may not be the same as the root
- * path for the other api calls.
+ * <p>IMPORTANT NOTE: Heartbeats have different api calls used to interact with them. The root path (/) may or may not
+ * be the same as the root path for the other api calls.
*
- * For example, performing these two calls: set_data("/path", data, acls); void set_worker_hb("/path", heartbeat, acls); may or may not
- * cause a collision in "/path". Never use the same paths with the *_hb* methods as you do with the others.
+ * <p>For example, performing these two calls: set_data("/path", data, acls); void set_worker_hb("/path", heartbeat,
+ * acls); may or may not cause a collision in "/path". Never use the same paths with the *_hb* methods as you do with
+ * the others.
*/
public interface IStateStorage extends Closeable {
@@ -200,7 +202,7 @@
void sync_path(String path);
/**
- * Allows us to delete the znodes within /storm/blobstore/key_name whose znodes start with the corresponding nimbusHostPortInfo
+ * Allows us to delete the znodes within /storm/blobstore/key_name whose znodes start with the corresponding nimbusHostPortInfo.
*
* @param path /storm/blobstore/key_name
* @param nimbusHostPortInfo Contains the host port information of a nimbus node.
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index c586dd1..94f0c86 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -142,6 +142,7 @@
List<String> errorTopologies();
/**
+ * Get backpressure topologies.
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
@@ -149,6 +150,7 @@
/**
* Get leader info from state store, which was written when a master gains leadership.
+ *
* <p>Caution: it can not be used for fencing and is only for informational purposes because we use ZK as our
* backend now, which could have a overdue info of nodes.
*
@@ -168,24 +170,28 @@
void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
/**
+ * Get topoloy backpressure.
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
boolean topologyBackpressure(String stormId, long timeoutMs, Runnable callback);
/**
+ * Setup backpressure.
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
void setupBackpressure(String stormId, Map<String, Object> topoConf);
/**
+ * Remove backpressure.
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
void removeBackpressure(String stormId);
/**
+ * Remove worker backpressure.
* @deprecated: In Storm 2.0. Retained for enabling transition from 1.x. Will be removed soon.
*/
@Deprecated
@@ -291,6 +297,7 @@
}
/**
+ * Get all supervisor info.
* @param callback be alerted if the list of supervisors change
* @return All of the supervisors with the ID as the key
*/
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index 277a150..401771a 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -149,8 +149,8 @@
while (true) {
try {
byte[] ret = null;
- int latest_time_secs = 0;
- boolean got_response = false;
+ int latestTimeSecs = 0;
+ boolean gotResponse = false;
HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
List<HBMessage> responses = pacemakerClientPool.sendAll(message);
@@ -160,18 +160,18 @@
continue;
}
// We got at least one GET_PULSE_RESPONSE message.
- got_response = true;
+ gotResponse = true;
byte[] details = response.get_data().get_pulse().get_details();
if (details == null) {
continue;
}
ClusterWorkerHeartbeat cwh = Utils.deserialize(details, ClusterWorkerHeartbeat.class);
- if (cwh != null && cwh.get_time_secs() > latest_time_secs) {
- latest_time_secs = cwh.get_time_secs();
+ if (cwh != null && cwh.get_time_secs() > latestTimeSecs) {
+ latestTimeSecs = cwh.get_time_secs();
ret = details;
}
}
- if (!got_response) {
+ if (!gotResponse) {
throw new WrappedHBExecutionException("Failed to get a response.");
}
return ret;
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
index 5e0cdd7..596aefa 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -24,10 +24,10 @@
public class PaceMakerStateStorageFactory implements StateStorageFactory {
@Override
- public IStateStorage mkStore(Map<String, Object> config, Map<String, Object> auth_conf, ClusterStateContext context) {
+ public IStateStorage mkStore(Map<String, Object> config, Map<String, Object> authConf, ClusterStateContext context) {
try {
ZKStateStorageFactory zkfact = new ZKStateStorageFactory();
- IStateStorage zkState = zkfact.mkStore(config, auth_conf, context);
+ IStateStorage zkState = zkfact.mkStore(config, authConf, context);
return new PaceMakerStateStorage(new PacemakerClientPool(config), zkState);
} catch (Exception e) {
throw Utils.wrapInRuntime(e);
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index ee530a0..fb321f7 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -16,5 +16,5 @@
public interface StateStorageFactory {
- IStateStorage mkStore(Map<String, Object> config, Map<String, Object> auth_conf, ClusterStateContext context);
+ IStateStorage mkStore(Map<String, Object> config, Map<String, Object> authConf, ClusterStateContext context);
}
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index f330278..80d5fb0 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -73,10 +73,10 @@
private ConcurrentHashMap<String, Runnable> credentialsCallback;
private ConcurrentHashMap<String, Runnable> logConfigCallback;
- public StormClusterStateImpl(IStateStorage StateStorage, ILocalAssignmentsBackend assignmentsassignmentsBackend,
+ public StormClusterStateImpl(IStateStorage stateStorage, ILocalAssignmentsBackend assignmentsassignmentsBackend,
ClusterStateContext context, boolean shouldCloseStateStorageOnDisconnect) throws Exception {
- this.stateStorage = StateStorage;
+ this.stateStorage = stateStorage;
this.shouldCloseStateStorageOnDisconnect = shouldCloseStateStorageOnDisconnect;
this.defaultAcls = context.getDefaultZkAcls();
this.context = context;
@@ -591,10 +591,7 @@
}
/**
- * To update this function due to APersistentMap/APersistentSet is clojure's structure
- *
- * @param stormId
- * @param newElems
+ * To update this function due to APersistentMap/APersistentSet is clojure's structure.
*/
@Override
public void updateStorm(String stormId, StormBase newElems) {
@@ -617,7 +614,7 @@
}
}
- Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+ Map<String, DebugOptions> componentDebug = new HashMap<>();
Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
@@ -639,10 +636,10 @@
DebugOptions debugOptions = new DebugOptions();
debugOptions.set_enable(enable);
debugOptions.set_samplingpct(samplingpct);
- ComponentDebug.put(key, debugOptions);
+ componentDebug.put(key, debugOptions);
}
- if (ComponentDebug.size() > 0) {
- newElems.set_component_debug(ComponentDebug);
+ if (componentDebug.size() > 0) {
+ newElems.set_component_debug(componentDebug);
}
if (StringUtils.isBlank(newElems.get_name())) {
@@ -746,13 +743,13 @@
@Override
public void reportError(String stormId, String componentId, String node, Long port, Throwable error) {
String path = ClusterUtils.errorPath(stormId, componentId);
- String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.stringifyError(error), Time.currentTimeSecs());
errorInfo.set_host(node);
errorInfo.set_port(port.intValue());
byte[] serData = Utils.serialize(errorInfo);
stateStorage.mkdirs(path, defaultAcls);
stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, defaultAcls);
+ String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
stateStorage.set_data(lastErrorPath, serData, defaultAcls);
List<String> childrens = stateStorage.get_children(path, false);
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index 2477f34..64ecf76 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -38,6 +38,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class ZKStateStorage implements IStateStorage {
private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
diff --git a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index 11feedc..ff22019 100644
--- a/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -21,12 +21,13 @@
import java.util.Map;
import org.apache.storm.utils.Utils;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class ZKStateStorageFactory implements StateStorageFactory {
@Override
- public IStateStorage mkStore(Map<String, Object> config, Map<String, Object> auth_conf, ClusterStateContext context) {
+ public IStateStorage mkStore(Map<String, Object> config, Map<String, Object> authConf, ClusterStateContext context) {
try {
- return new ZKStateStorage(config, auth_conf, context);
+ return new ZKStateStorage(config, authConf, context);
} catch (Exception e) {
throw Utils.wrapInRuntime(e);
}
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
index 59ff7e6..67c0482 100644
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java
@@ -37,7 +37,7 @@
}
- public synchronized static CgroupCenter getInstance() {
+ public static synchronized CgroupCenter getInstance() {
if (CgroupUtils.enabled()) {
instance = new CgroupCenter();
return instance;
@@ -82,10 +82,10 @@
if (type == null) {
continue;
}
- int hierarchyID = Integer.valueOf(split[1]);
+ int hierarchyId = Integer.valueOf(split[1]);
int cgroupNum = Integer.valueOf(split[2]);
boolean enable = Integer.valueOf(split[3]).intValue() == 1 ? true : false;
- subSystems.add(new SubSystem(type, hierarchyID, cgroupNum, enable));
+ subSystems.add(new SubSystem(type, hierarchyId, cgroupNum, enable));
}
return subSystems;
} catch (Exception e) {
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
index 082989d..07f1abe 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java
@@ -45,9 +45,6 @@
this.isRoot = false;
}
- /**
- * rootCgroup
- */
public CgroupCommon(Hierarchy hierarchy, String dir) {
this.name = "";
this.hierarchy = hierarchy;
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
index c73d1f1..63802a0 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java
@@ -18,61 +18,61 @@
public interface CgroupCommonOperation {
/**
- * add task to cgroup
+ * add task to cgroup.
*
* @param taskid task id of task to add
*/
public void addTask(int taskid) throws IOException;
/**
- * Get a list of task ids running in CGroup
+ * Get a list of task ids running in CGroup.
*/
public Set<Integer> getTasks() throws IOException;
/**
- * add a process to cgroup
+ * add a process to cgroup.
*
* @param pid the PID of the process to add
*/
public void addProcs(int pid) throws IOException;
/**
- * get the PIDs of processes running in cgroup
+ * get the PIDs of processes running in cgroup.
*/
public Set<Long> getPids() throws IOException;
/**
- * to get the notify_on_release config
+ * to get the notify_on_release config.
*/
public boolean getNotifyOnRelease() throws IOException;
/**
- * to set notify_on_release config in cgroup
+ * to set notify_on_release config in cgroup.
*/
public void setNotifyOnRelease(boolean flag) throws IOException;
/**
- * get the command for the relase agent to execute
+ * get the command for the relase agent to execute.
*/
public String getReleaseAgent() throws IOException;
/**
- * set a command for the release agent to execute
+ * set a command for the release agent to execute.
*/
public void setReleaseAgent(String command) throws IOException;
/**
- * get the cgroup.clone_children config
+ * get the cgroup.clone_children config.
*/
public boolean getCgroupCloneChildren() throws IOException;
/**
- * Set the cgroup.clone_children config
+ * Set the cgroup.clone_children config.
*/
public void setCgroupCloneChildren(boolean flag) throws IOException;
/**
- * set event control config
+ * set event control config.
*/
public void setEventControl(String eventFd, String controlFd, String... args) throws IOException;
}
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
index 7ec2b47..6387fe3 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java
@@ -23,52 +23,52 @@
public interface CgroupOperation {
/**
- * Get a list of hierarchies
+ * Get a list of hierarchies.
*/
public List<Hierarchy> getHierarchies();
/**
- * get a list of available subsystems
+ * get a list of available subsystems.
*/
public Set<SubSystem> getSubSystems();
/**
- * Check if a subsystem is enabled
+ * Check if a subsystem is enabled.
*/
public boolean isSubSystemEnabled(SubSystemType subsystem);
/**
- * get the first hierarchy that has a certain subsystem isMounted
+ * get the first hierarchy that has a certain subsystem isMounted.
*/
public Hierarchy getHierarchyWithSubSystem(SubSystemType subsystem);
/**
- * get the first hierarchy that has a certain list of subsystems isMounted
+ * get the first hierarchy that has a certain list of subsystems isMounted.
*/
public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems);
/**
- * check if a hiearchy is mounted
+ * check if a hiearchy is mounted.
*/
public boolean isMounted(Hierarchy hierarchy);
/**
- * mount a hierarchy
+ * mount a hierarchy.
*/
public void mount(Hierarchy hierarchy) throws IOException;
/**
- * umount a heirarchy
+ * umount a heirarchy.
*/
public void umount(Hierarchy hierarchy) throws IOException;
/**
- * create a cgroup
+ * create a cgroup.
*/
public void createCgroup(CgroupCommon cgroup) throws SecurityException;
/**
- * delete a cgroup
+ * delete a cgroup.
*/
public void deleteCgroup(CgroupCommon cgroup) throws IOException;
}
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
index 2990743..d55361e 100644
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java
@@ -48,7 +48,7 @@
}
/**
- * Get a set of SubSystemType objects from a comma delimited list of subsystem names
+ * Get a set of SubSystemType objects from a comma delimited list of subsystem names.
*/
public static Set<SubSystemType> getSubSystemsFromString(String str) {
Set<SubSystemType> result = new HashSet<SubSystemType>();
@@ -64,7 +64,7 @@
}
/**
- * Get a string that is a comma delimited list of subsystems
+ * Get a string that is a comma delimited list of subsystems.
*/
public static String subSystemsToString(Set<SubSystemType> subSystems) {
StringBuilder sb = new StringBuilder();
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
index 9dfc15a..a949e87 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/Device.java
@@ -13,7 +13,7 @@
package org.apache.storm.container.cgroup;
/**
- * a class that represents a device in linux
+ * a class that represents a device in linux.
*/
public class Device {
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
index ea79918..7ef5296 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java
@@ -15,7 +15,7 @@
import java.util.Set;
/**
- * A class that describes a cgroup hierarchy
+ * A class that describes a cgroup hierarchy.
*/
public class Hierarchy {
@@ -38,14 +38,14 @@
}
/**
- * get subsystems
+ * get subsystems.
*/
public Set<SubSystemType> getSubSystems() {
return subSystems;
}
/**
- * get all subsystems in hierarchy as a comma delimited list
+ * get all subsystems in hierarchy as a comma delimited list.
*/
public String getType() {
return type;
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
index 8a02584..f5aa32f 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystem.java
@@ -13,21 +13,22 @@
package org.apache.storm.container.cgroup;
/**
- * a class that implements operations that can be performed on a cgroup subsystem
+ * a class that implements operations that can be performed on a cgroup subsystem.
*/
public class SubSystem {
private SubSystemType type;
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private int hierarchyID;
private int cgroupsNum;
private boolean enable;
- public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, boolean enable) {
+ public SubSystem(SubSystemType type, int hierarchyId, int cgroupNum, boolean enable) {
this.type = type;
- this.hierarchyID = hierarchyID;
+ this.hierarchyID = hierarchyId;
this.cgroupsNum = cgroupNum;
this.enable = enable;
}
@@ -40,10 +41,12 @@
this.type = type;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public int getHierarchyID() {
return hierarchyID;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void setHierarchyID(int hierarchyID) {
this.hierarchyID = hierarchyID;
}
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
index ff8ab28..f2ea9dd 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java
@@ -13,7 +13,7 @@
package org.apache.storm.container.cgroup;
/**
- * A enum class to described the subsystems that can be used
+ * A enum class to described the subsystems that can be used.
*/
public enum SubSystemType {
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
index 17a1a71..a03fb52 100644
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java
@@ -19,7 +19,7 @@
import org.slf4j.LoggerFactory;
/**
- * A class that implements system operations for using cgroups
+ * A class that implements system operations for using cgroups.
*/
public class SystemOperation {
@@ -33,13 +33,13 @@
public static void mount(String name, String target, String type, String options) throws IOException {
StringBuilder sb = new StringBuilder();
sb.append("mount -t ")
- .append(type)
- .append(" -o ")
- .append(options)
- .append(" ")
- .append(name)
- .append(" ")
- .append(target);
+ .append(type)
+ .append(" -o ")
+ .append(options)
+ .append(" ")
+ .append(name)
+ .append(" ")
+ .append(target);
SystemOperation.exec(sb.toString());
}
diff --git a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
index 75fe134..7bcf570 100755
--- a/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
+++ b/storm-client/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java
@@ -95,26 +95,32 @@
return parseConfig(BLKIO_THROTTLE_WRITE_BPS_DEVICE);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void setReadIOps(Device device, long iops) throws IOException {
CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), makeContext(device, iops));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Long> getReadIOps() throws IOException {
return parseConfig(BLKIO_THROTTLE_READ_IOPS_DEVICE);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void setWriteIOps(Device device, long iops) throws IOException {
CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), makeContext(device, iops));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Long> getWriteIOps() throws IOException {
return parseConfig(BLKIO_THROTTLE_WRITE_IOPS_DEVICE);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getThrottleIOServiced() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICED)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getThrottleIOServiceByte() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES)));
}
@@ -127,26 +133,32 @@
return parseConfig(BLKIO_SECTORS);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOServiced() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICED)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOServiceBytes() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_BYTES)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOServiceTime() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_TIME)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOWaitTime() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_WAIT_TIME)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOMerged() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_MERGED)));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Map<Device, Map<RecordType, Long>> getIOQueued() throws IOException {
return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_QUEUED)));
}
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
index 9f733b1..4b7fcf4 100644
--- a/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchBoltExecutor.java
@@ -29,22 +29,22 @@
public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCallback {
public static final Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class);
- byte[] _boltSer;
- Map<Object, IBatchBolt> _openTransactions;
- Map _conf;
- TopologyContext _context;
- BatchOutputCollectorImpl _collector;
+ private byte[] boltSer;
+ private Map<Object, IBatchBolt> openTransactions;
+ private Map conf;
+ private TopologyContext context;
+ private BatchOutputCollectorImpl collector;
public BatchBoltExecutor(IBatchBolt bolt) {
- _boltSer = Utils.javaSerialize(bolt);
+ boltSer = Utils.javaSerialize(bolt);
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context, OutputCollector collector) {
- _conf = conf;
- _context = context;
- _collector = new BatchOutputCollectorImpl(collector);
- _openTransactions = new HashMap<>();
+ this.conf = conf;
+ this.context = context;
+ this.collector = new BatchOutputCollectorImpl(collector);
+ openTransactions = new HashMap<>();
}
@Override
@@ -53,10 +53,10 @@
IBatchBolt bolt = getBatchBolt(id);
try {
bolt.execute(input);
- _collector.ack(input);
+ collector.ack(input);
} catch (FailedException e) {
LOG.error("Failed to process tuple in batch", e);
- _collector.fail(input);
+ collector.fail(input);
}
}
@@ -67,13 +67,13 @@
@Override
public void finishedId(Object id) {
IBatchBolt bolt = getBatchBolt(id);
- _openTransactions.remove(id);
+ openTransactions.remove(id);
bolt.finishBatch();
}
@Override
public void timeoutId(Object attempt) {
- _openTransactions.remove(attempt);
+ openTransactions.remove(attempt);
}
@@ -88,16 +88,16 @@
}
private IBatchBolt getBatchBolt(Object id) {
- IBatchBolt bolt = _openTransactions.get(id);
+ IBatchBolt bolt = openTransactions.get(id);
if (bolt == null) {
bolt = newTransactionalBolt();
- bolt.prepare(_conf, _context, _collector, id);
- _openTransactions.put(id, bolt);
+ bolt.prepare(conf, context, collector, id);
+ openTransactions.put(id, bolt);
}
return bolt;
}
private IBatchBolt newTransactionalBolt() {
- return Utils.javaDeserialize(_boltSer, IBatchBolt.class);
+ return Utils.javaDeserialize(boltSer, IBatchBolt.class);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
index fb57e4a..2baff31 100644
--- a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollector.java
@@ -43,7 +43,7 @@
public abstract void emitDirect(int taskId, String streamId, List<Object> tuple);
/**
- * Flush any buffered tuples (when batching is enabled)
+ * Flush any buffered tuples (when batching is enabled).
*/
public abstract void flush();
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
index b4b00c4..bea8a45 100644
--- a/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/coordination/BatchOutputCollectorImpl.java
@@ -17,37 +17,37 @@
import org.apache.storm.tuple.Tuple;
public class BatchOutputCollectorImpl extends BatchOutputCollector {
- OutputCollector _collector;
+ private OutputCollector collector;
public BatchOutputCollectorImpl(OutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
public List<Integer> emit(String streamId, List<Object> tuple) {
- return _collector.emit(streamId, tuple);
+ return collector.emit(streamId, tuple);
}
@Override
public void emitDirect(int taskId, String streamId, List<Object> tuple) {
- _collector.emitDirect(taskId, streamId, tuple);
+ collector.emitDirect(taskId, streamId, tuple);
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void reportError(Throwable error) {
- _collector.reportError(error);
+ collector.reportError(error);
}
public void ack(Tuple tup) {
- _collector.ack(tup);
+ collector.ack(tup);
}
public void fail(Tuple tup) {
- _collector.fail(tup);
+ collector.fail(tup);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java b/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
index 181e4ec..18a14db 100644
--- a/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/coordination/CoordinatedBolt.java
@@ -43,13 +43,13 @@
*/
public class CoordinatedBolt implements IRichBolt {
public static final Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class);
- private Map<String, SourceArgs> _sourceArgs;
- private IdStreamSpec _idStreamSpec;
- private IRichBolt _delegate;
- private Integer _numSourceReports;
- private List<Integer> _countOutTasks = new ArrayList<>();
- private OutputCollector _collector;
- private TimeCacheMap<Object, TrackingInfo> _tracked;
+ private Map<String, SourceArgs> sourceArgs;
+ private IdStreamSpec idStreamSpec;
+ private IRichBolt delegate;
+ private Integer numSourceReports;
+ private List<Integer> countOutTasks = new ArrayList<>();
+ private OutputCollector collector;
+ private TimeCacheMap<Object, TrackingInfo> tracked;
public CoordinatedBolt(IRichBolt delegate) {
this(delegate, null, null);
@@ -60,12 +60,12 @@
}
public CoordinatedBolt(IRichBolt delegate, Map<String, SourceArgs> sourceArgs, IdStreamSpec idStreamSpec) {
- _sourceArgs = sourceArgs;
- if (_sourceArgs == null) {
- _sourceArgs = new HashMap<>();
+ this.sourceArgs = sourceArgs;
+ if (this.sourceArgs == null) {
+ this.sourceArgs = new HashMap<>();
}
- _delegate = delegate;
- _idStreamSpec = idStreamSpec;
+ this.delegate = delegate;
+ this.idStreamSpec = idStreamSpec;
}
private static Map<String, SourceArgs> singleSourceArgs(String sourceComponent, SourceArgs sourceArgs) {
@@ -77,27 +77,27 @@
@Override
public void prepare(Map<String, Object> config, TopologyContext context, OutputCollector collector) {
TimeCacheMap.ExpiredCallback<Object, TrackingInfo> callback = null;
- if (_delegate instanceof TimeoutCallback) {
+ if (delegate instanceof TimeoutCallback) {
callback = new TimeoutItems();
}
- _tracked = new TimeCacheMap<>(context.maxTopologyMessageTimeout(), callback);
- _collector = collector;
- _delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector)));
+ tracked = new TimeCacheMap<>(context.maxTopologyMessageTimeout(), callback);
+ this.collector = collector;
+ delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector)));
for (String component : Utils.get(context.getThisTargets(),
Constants.COORDINATED_STREAM_ID,
new HashMap<String, Grouping>())
.keySet()) {
for (Integer task : context.getComponentTasks(component)) {
- _countOutTasks.add(task);
+ countOutTasks.add(task);
}
}
- if (!_sourceArgs.isEmpty()) {
- _numSourceReports = 0;
- for (Entry<String, SourceArgs> entry : _sourceArgs.entrySet()) {
+ if (!sourceArgs.isEmpty()) {
+ numSourceReports = 0;
+ for (Entry<String, SourceArgs> entry : sourceArgs.entrySet()) {
if (entry.getValue().singleCount) {
- _numSourceReports += 1;
+ numSourceReports += 1;
} else {
- _numSourceReports += context.getComponentTasks(entry.getKey()).size();
+ numSourceReports += context.getComponentTasks(entry.getKey()).size();
}
}
}
@@ -107,61 +107,59 @@
Object id = tup.getValue(0);
boolean failed = false;
- synchronized (_tracked) {
- TrackingInfo track = _tracked.get(id);
+ synchronized (tracked) {
+ TrackingInfo track = tracked.get(id);
try {
if (track != null) {
boolean delayed = false;
- if (_idStreamSpec == null && type == TupleType.COORD || _idStreamSpec != null && type == TupleType.ID) {
+ if (idStreamSpec == null && type == TupleType.COORD || idStreamSpec != null && type == TupleType.ID) {
track.ackTuples.add(tup);
delayed = true;
}
if (track.failed) {
failed = true;
for (Tuple t : track.ackTuples) {
- _collector.fail(t);
+ collector.fail(t);
}
- _tracked.remove(id);
- } else if (track.receivedId
- && (_sourceArgs.isEmpty() ||
- track.reportCount == _numSourceReports &&
- track.expectedTupleCount == track.receivedTuples)) {
- if (_delegate instanceof FinishedCallback) {
- ((FinishedCallback) _delegate).finishedId(id);
+ tracked.remove(id);
+ } else if (track.receivedId && (sourceArgs.isEmpty()
+ || track.reportCount == numSourceReports && track.expectedTupleCount == track.receivedTuples)) {
+ if (delegate instanceof FinishedCallback) {
+ ((FinishedCallback) delegate).finishedId(id);
}
- if (!(_sourceArgs.isEmpty() || type != TupleType.REGULAR)) {
+ if (!(sourceArgs.isEmpty() || type != TupleType.REGULAR)) {
throw new IllegalStateException("Coordination condition met on a non-coordinating tuple. Should be impossible");
}
- Iterator<Integer> outTasks = _countOutTasks.iterator();
+ Iterator<Integer> outTasks = countOutTasks.iterator();
while (outTasks.hasNext()) {
int task = outTasks.next();
int numTuples = Utils.get(track.taskEmittedTuples, task, 0);
- _collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tup, new Values(id, numTuples));
+ collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tup, new Values(id, numTuples));
}
for (Tuple t : track.ackTuples) {
- _collector.ack(t);
+ collector.ack(t);
}
track.finished = true;
- _tracked.remove(id);
+ tracked.remove(id);
}
if (!delayed && type != TupleType.REGULAR) {
if (track.failed) {
- _collector.fail(tup);
+ collector.fail(tup);
} else {
- _collector.ack(tup);
+ collector.ack(tup);
}
}
} else {
if (type != TupleType.REGULAR) {
- _collector.fail(tup);
+ collector.fail(tup);
}
}
} catch (FailedException e) {
LOG.error("Failed to finish batch", e);
for (Tuple t : track.ackTuples) {
- _collector.fail(t);
+ collector.fail(t);
}
- _tracked.remove(id);
+ tracked.remove(id);
failed = true;
}
}
@@ -173,58 +171,58 @@
Object id = tuple.getValue(0);
TrackingInfo track;
TupleType type = getTupleType(tuple);
- synchronized (_tracked) {
- track = _tracked.get(id);
+ synchronized (tracked) {
+ track = tracked.get(id);
if (track == null) {
track = new TrackingInfo();
- if (_idStreamSpec == null) {
+ if (idStreamSpec == null) {
track.receivedId = true;
}
- _tracked.put(id, track);
+ tracked.put(id, track);
}
}
if (type == TupleType.ID) {
- synchronized (_tracked) {
+ synchronized (tracked) {
track.receivedId = true;
}
checkFinishId(tuple, type);
} else if (type == TupleType.COORD) {
int count = (Integer) tuple.getValue(1);
- synchronized (_tracked) {
+ synchronized (tracked) {
track.reportCount++;
track.expectedTupleCount += count;
}
checkFinishId(tuple, type);
} else {
- synchronized (_tracked) {
- _delegate.execute(tuple);
+ synchronized (tracked) {
+ delegate.execute(tuple);
}
}
}
@Override
public void cleanup() {
- _delegate.cleanup();
- _tracked.cleanup();
+ delegate.cleanup();
+ tracked.cleanup();
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _delegate.declareOutputFields(declarer);
+ delegate.declareOutputFields(declarer);
declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, new Fields("id", "count"));
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _delegate.getComponentConfiguration();
+ return delegate.getComponentConfiguration();
}
private TupleType getTupleType(Tuple tuple) {
- if (_idStreamSpec != null
- && tuple.getSourceGlobalStreamId().equals(_idStreamSpec._id)) {
+ if (idStreamSpec != null
+ && tuple.getSourceGlobalStreamId().equals(idStreamSpec.id)) {
return TupleType.ID;
- } else if (!_sourceArgs.isEmpty()
+ } else if (!sourceArgs.isEmpty()
&& tuple.getSourceStreamId().equals(Constants.COORDINATED_STREAM_ID)) {
return TupleType.COORD;
} else {
@@ -279,19 +277,19 @@
@Override
public String toString() {
- return "reportCount: " + reportCount + "\n" +
- "expectedTupleCount: " + expectedTupleCount + "\n" +
- "receivedTuples: " + receivedTuples + "\n" +
- "failed: " + failed + "\n" +
- taskEmittedTuples.toString();
+ return "reportCount: " + reportCount + "\n"
+ + "expectedTupleCount: " + expectedTupleCount + "\n"
+ + "receivedTuples: " + receivedTuples + "\n"
+ + "failed: " + failed + "\n"
+ + taskEmittedTuples.toString();
}
}
public static class IdStreamSpec implements Serializable {
- GlobalStreamId _id;
+ GlobalStreamId id;
protected IdStreamSpec(String component, String stream) {
- _id = new GlobalStreamId(component, stream);
+ id = new GlobalStreamId(component, stream);
}
public static IdStreamSpec makeDetectSpec(String component, String stream) {
@@ -299,20 +297,20 @@
}
public GlobalStreamId getGlobalStreamId() {
- return _id;
+ return id;
}
}
public class CoordinatedOutputCollector implements IOutputCollector {
- IOutputCollector _delegate;
+ IOutputCollector delegate;
public CoordinatedOutputCollector(IOutputCollector delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
@Override
public List<Integer> emit(String stream, Collection<Tuple> anchors, List<Object> tuple) {
- List<Integer> tasks = _delegate.emit(stream, anchors, tuple);
+ List<Integer> tasks = delegate.emit(stream, anchors, tuple);
updateTaskCounts(tuple.get(0), tasks);
return tasks;
}
@@ -320,58 +318,58 @@
@Override
public void emitDirect(int task, String stream, Collection<Tuple> anchors, List<Object> tuple) {
updateTaskCounts(tuple.get(0), Arrays.asList(task));
- _delegate.emitDirect(task, stream, anchors, tuple);
+ delegate.emitDirect(task, stream, anchors, tuple);
}
@Override
public void ack(Tuple tuple) {
Object id = tuple.getValue(0);
- synchronized (_tracked) {
- TrackingInfo track = _tracked.get(id);
+ synchronized (tracked) {
+ TrackingInfo track = tracked.get(id);
if (track != null) {
track.receivedTuples++;
}
}
boolean failed = checkFinishId(tuple, TupleType.REGULAR);
if (failed) {
- _delegate.fail(tuple);
+ delegate.fail(tuple);
} else {
- _delegate.ack(tuple);
+ delegate.ack(tuple);
}
}
@Override
public void fail(Tuple tuple) {
Object id = tuple.getValue(0);
- synchronized (_tracked) {
- TrackingInfo track = _tracked.get(id);
+ synchronized (tracked) {
+ TrackingInfo track = tracked.get(id);
if (track != null) {
track.failed = true;
}
}
checkFinishId(tuple, TupleType.REGULAR);
- _delegate.fail(tuple);
+ delegate.fail(tuple);
}
@Override
public void flush() {
- _delegate.flush();
+ delegate.flush();
}
@Override
public void resetTimeout(Tuple tuple) {
- _delegate.resetTimeout(tuple);
+ delegate.resetTimeout(tuple);
}
@Override
public void reportError(Throwable error) {
- _delegate.reportError(error);
+ delegate.reportError(error);
}
private void updateTaskCounts(Object id, List<Integer> tasks) {
- synchronized (_tracked) {
- TrackingInfo track = _tracked.get(id);
+ synchronized (tracked) {
+ TrackingInfo track = tracked.get(id);
if (track != null) {
Map<Integer, Integer> taskEmittedTuples = track.taskEmittedTuples;
for (Integer task : tasks) {
@@ -386,12 +384,12 @@
private class TimeoutItems implements TimeCacheMap.ExpiredCallback<Object, TrackingInfo> {
@Override
public void expire(Object id, TrackingInfo val) {
- synchronized (_tracked) {
+ synchronized (tracked) {
// the combination of the lock and the finished flag ensure that
// an id is never timed out if it has been finished
val.failed = true;
if (!val.finished) {
- ((TimeoutCallback) _delegate).timeoutId(id);
+ ((TimeoutCallback) delegate).timeoutId(id);
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
index 27a54e8..0f83597 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java
@@ -121,7 +121,7 @@
}
/**
- * A bridge between CustomStreamGrouping and LoadAwareCustomStreamGrouping
+ * A bridge between CustomStreamGrouping and LoadAwareCustomStreamGrouping.
*/
public static class BasicLoadAwareCustomStreamGrouping implements LoadAwareCustomStreamGrouping {
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
index 2bb1871..6a5cee8 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/StormCommon.java
@@ -252,9 +252,6 @@
@SuppressWarnings("unchecked")
public static void addAcker(Map<String, Object> conf, StormTopology topology) {
- int ackerNum =
- ObjectReader.getInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS), ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
- Map<GlobalStreamId, Grouping> inputs = ackerInputs(topology);
Map<String, StreamInfo> outputStreams = new HashMap<String, StreamInfo>();
outputStreams.put(Acker.ACKER_ACK_STREAM_ID, Thrift.directOutputFields(Arrays.asList("id", "time-delta-ms")));
@@ -262,9 +259,12 @@
outputStreams.put(Acker.ACKER_RESET_TIMEOUT_STREAM_ID, Thrift.directOutputFields(Arrays.asList("id", "time-delta-ms")));
Map<String, Object> ackerConf = new HashMap<>();
+ int ackerNum =
+ ObjectReader.getInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS), ObjectReader.getInt(conf.get(Config.TOPOLOGY_WORKERS)));
ackerConf.put(Config.TOPOLOGY_TASKS, ackerNum);
ackerConf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, ObjectReader.getInt(conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)));
+ Map<GlobalStreamId, Grouping> inputs = ackerInputs(topology);
Bolt acker = Thrift.prepareSerializedBoltDetails(inputs, makeAckerBolt(), outputStreams, ackerNum, ackerConf);
for (Bolt bolt : topology.get_bolts().values()) {
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
index 79b5af6..814ec12 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/AdvancedFSOps.java
@@ -43,16 +43,17 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AdvancedFSOps implements IAdvancedFSOps {
private static final Logger LOG = LoggerFactory.getLogger(AdvancedFSOps.class);
- protected final boolean _symlinksDisabled;
+ protected final boolean symlinksDisabled;
protected AdvancedFSOps(Map<String, Object> conf) {
- _symlinksDisabled = (boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false);
+ symlinksDisabled = (boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false);
}
/**
- * Factory to create a new AdvancedFSOps
+ * Factory to create a new AdvancedFSOps.
*
* @param conf the configuration of the process
* @return the appropriate instance of the class for this config and environment.
@@ -68,7 +69,7 @@
}
/**
- * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)--- On some systems that do not support this, it may become a noop
+ * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)--- On some systems that do not support this, it may become a noop.
*
* @param dir the directory to change permissions on
* @throws IOException on any error
@@ -83,7 +84,7 @@
}
/**
- * Move fromDir to toDir, and try to make it an atomic move if possible
+ * Move fromDir to toDir, and try to make it an atomic move if possible.
*
* @param fromDir what to move
* @param toDir where to move it from
@@ -96,7 +97,8 @@
}
/**
- * @return true if an atomic directory move works, else false.
+ * Check whether supports atomic directory move.
+ * @return true if an atomic directory move works, else false
*/
@Override
public boolean supportsAtomicDirectoryMove() {
@@ -104,7 +106,7 @@
}
/**
- * Copy a directory
+ * Copy a directory.
*
* @param fromDir from where
* @param toDir to where
@@ -116,7 +118,7 @@
}
/**
- * Setup permissions properly for an internal blob store path
+ * Setup permissions properly for an internal blob store path.
*
* @param path the path to set the permissions on
* @param user the user to change the permissions for
@@ -155,12 +157,13 @@
try {
FileUtils.forceDelete(path);
} catch (FileNotFoundException ignored) {
+ //ignore
}
}
}
/**
- * Setup the permissions for the storm code dir
+ * Setup the permissions for the storm code dir.
*
* @param user the user that owns the topology
* @param path the directory to set the permissions on
@@ -172,7 +175,7 @@
}
/**
- * Setup the permissions for the worker artifacts dirs
+ * Setup the permissions for the worker artifacts dirs.
*
* @param user the user that owns the topology
* @param path the directory to set the permissions on
@@ -230,7 +233,7 @@
}
/**
- * Check if a file exists or not
+ * Check if a file exists or not.
*
* @param path the path to check
* @return true if it exists else false
@@ -243,7 +246,7 @@
}
/**
- * Check if a file exists or not
+ * Check if a file exists or not.
*
* @param path the path to check
* @return true if it exists else false
@@ -256,7 +259,7 @@
}
/**
- * Get a writer for the given location
+ * Get a writer for the given location.
*
* @param file the file to write to
* @return the Writer to use.
@@ -269,7 +272,7 @@
}
/**
- * Get an output stream to write to a given file
+ * Get an output stream to write to a given file.
*
* @param file the file to write to
* @return an OutputStream for that file
@@ -282,7 +285,7 @@
}
/**
- * Dump a string to a file
+ * Dump a string to a file.
*
* @param location where to write to
* @param data the data to write
@@ -300,7 +303,7 @@
}
/**
- * Read the contents of a file into a String
+ * Read the contents of a file into a String.
*
* @param location the file to read
* @return the contents of the file
@@ -326,7 +329,7 @@
}
/**
- * Create a symbolic link pointing at target
+ * Create a symbolic link pointing at target.
*
* @param link the link to create
* @param target where it should point to
@@ -334,7 +337,7 @@
*/
@Override
public void createSymlink(File link, File target) throws IOException {
- if (_symlinksDisabled) {
+ if (symlinksDisabled) {
throw new IOException("Symlinks have been disabled, this should not be called");
}
Path plink = link.toPath().toAbsolutePath();
@@ -350,21 +353,22 @@
Files.createSymbolicLink(plink, ptarget);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static class AdvancedRunAsUserFSOps extends AdvancedFSOps {
- private final Map<String, Object> _conf;
+ private final Map<String, Object> conf;
public AdvancedRunAsUserFSOps(Map<String, Object> conf) {
super(conf);
if (Utils.isOnWindows()) {
throw new UnsupportedOperationException("ERROR: Windows doesn't support running workers as different users yet");
}
- _conf = conf;
+ this.conf = conf;
}
@Override
public void setupBlobPermissions(File path, String user) throws IOException {
String logPrefix = "setup blob permissions for " + path;
- ClientSupervisorUtils.processLauncherAndWait(_conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
+ ClientSupervisorUtils.processLauncherAndWait(conf, user, Arrays.asList("blob", path.toString()), null, logPrefix);
}
@Override
@@ -378,7 +382,7 @@
List<String> commands = new ArrayList<>();
commands.add("rmr");
commands.add(absolutePath);
- ClientSupervisorUtils.processLauncherAndWait(_conf, user, commands, null, logPrefix);
+ ClientSupervisorUtils.processLauncherAndWait(conf, user, commands, null, logPrefix);
if (Utils.checkFileExists(absolutePath)) {
// It's possible that permissions were not set properly on the directory, and
@@ -399,18 +403,19 @@
@Override
public void setupStormCodeDir(String user, File path) throws IOException {
- ClientSupervisorUtils.setupStormCodeDir(_conf, user, path.getCanonicalPath());
+ ClientSupervisorUtils.setupStormCodeDir(conf, user, path.getCanonicalPath());
}
@Override
public void setupWorkerArtifactsDir(String user, File path) throws IOException {
- ClientSupervisorUtils.setupWorkerArtifactsDir(_conf, user, path.getCanonicalPath());
+ ClientSupervisorUtils.setupWorkerArtifactsDir(conf, user, path.getCanonicalPath());
}
}
/**
- * Operations that need to override the default ones when running on Windows
+ * Operations that need to override the default ones when running on Windows.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static class AdvancedWindowsFSOps extends AdvancedFSOps {
public AdvancedWindowsFSOps(Map<String, Object> conf) {
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
index 0aa5734..6bf6752 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ClientSupervisorUtils.java
@@ -42,7 +42,6 @@
static boolean doRequiredTopoFilesExist(Map<String, Object> conf, String stormId) throws IOException {
String stormroot = ConfigUtils.supervisorStormDistRoot(conf, stormId);
- String stormjarpath = ConfigUtils.supervisorStormJarPath(stormroot);
String stormcodepath = ConfigUtils.supervisorStormCodePath(stormroot);
String stormconfpath = ConfigUtils.supervisorStormConfPath(stormroot);
if (!Utils.checkFileExists(stormroot)) {
@@ -54,6 +53,7 @@
if (!Utils.checkFileExists(stormconfpath)) {
return false;
}
+ String stormjarpath = ConfigUtils.supervisorStormJarPath(stormroot);
if (ConfigUtils.isLocalMode(conf) || Utils.checkFileExists(stormjarpath)) {
return true;
}
@@ -111,9 +111,6 @@
* @param exitCodeCallback code to be called passing the exit code value when the process completes
* @param dir the working directory of the new process
* @return the new process
- *
- * @throws IOException
- * @see ProcessBuilder
*/
public static Process launchProcess(List<String> command,
Map<String, String> environment,
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
index 9c18ce1..de42895 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/ExitCodeCallback.java
@@ -18,7 +18,7 @@
public interface ExitCodeCallback {
/**
- * The process finished
+ * The process finished.
*
* @param exitCode the exit code of the finished process.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
index cb5dfc6..f55ba7f 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/supervisor/IAdvancedFSOps.java
@@ -18,7 +18,6 @@
package org.apache.storm.daemon.supervisor;
-
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
@@ -27,10 +26,11 @@
import java.nio.file.Path;
import java.util.Map;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface IAdvancedFSOps {
/**
- * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)--- On some systems that do not support this, it may become a noop
+ * Set directory permissions to (OWNER)RWX (GROUP)R-X (OTHER)--- On some systems that do not support this, it may become a noop.
*
* @param dir the directory to change permissions on
* @throws IOException on any error
@@ -38,7 +38,7 @@
void restrictDirectoryPermissions(File dir) throws IOException;
/**
- * Move fromDir to toDir, and try to make it an atomic move if possible
+ * Move fromDir to toDir, and try to make it an atomic move if possible.
*
* @param fromDir what to move
* @param toDir where to move it from
@@ -47,12 +47,13 @@
void moveDirectoryPreferAtomic(File fromDir, File toDir) throws IOException;
/**
- * @return true if an atomic directory move works, else false.
+ * Check whether supports atomic directory move.
+ * @return true if an atomic directory move works, else false
*/
boolean supportsAtomicDirectoryMove();
/**
- * Copy a directory
+ * Copy a directory.
*
* @param fromDir from where
* @param toDir to where
@@ -61,7 +62,7 @@
void copyDirectory(File fromDir, File toDir) throws IOException;
/**
- * Setup permissions properly for an internal blob store path
+ * Setup permissions properly for an internal blob store path.
*
* @param path the path to set the permissions on
* @param user the user to change the permissions for
@@ -88,7 +89,7 @@
void deleteIfExists(File path) throws IOException;
/**
- * Setup the permissions for the storm code dir
+ * Setup the permissions for the storm code dir.
*
* @param user the owner of the topology
* @param path the directory to set the permissions on
@@ -97,7 +98,7 @@
void setupStormCodeDir(String user, File path) throws IOException;
/**
- * Setup the permissions for the worker artifacts dirs
+ * Setup the permissions for the worker artifacts dirs.
*
* @param user the owner of the topology
* @param path the directory to set the permissions on
@@ -154,7 +155,7 @@
DirectoryStream<Path> newDirectoryStream(Path dir) throws IOException;
/**
- * Check if a file exists or not
+ * Check if a file exists or not.
*
* @param path the path to check
* @return true if it exists else false
@@ -164,7 +165,7 @@
boolean fileExists(File path) throws IOException;
/**
- * Check if a file exists or not
+ * Check if a file exists or not.
*
* @param path the path to check
* @return true if it exists else false
@@ -174,7 +175,7 @@
boolean fileExists(Path path) throws IOException;
/**
- * Get a writer for the given location
+ * Get a writer for the given location.
*
* @param file the file to write to
* @return the Writer to use.
@@ -184,7 +185,7 @@
Writer getWriter(File file) throws IOException;
/**
- * Get an output stream to write to a given file
+ * Get an output stream to write to a given file.
*
* @param file the file to write to
* @return an OutputStream for that file
@@ -194,7 +195,7 @@
OutputStream getOutputStream(File file) throws IOException;
/**
- * Dump a string to a file
+ * Dump a string to a file.
*
* @param location where to write to
* @param data the data to write
@@ -203,7 +204,7 @@
void dump(File location, String data) throws IOException;
/**
- * Read the contents of a file into a String
+ * Read the contents of a file into a String.
*
* @param location the file to read
* @return the contents of the file
@@ -223,7 +224,7 @@
byte[] slurp(File location) throws IOException;
/**
- * Create a symbolic link pointing at target
+ * Create a symbolic link pointing at target.
*
* @param link the link to create
* @param target where it should point to
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/BackPressureTracker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/BackPressureTracker.java
index 7e98658..5800a00 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/BackPressureTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/BackPressureTracker.java
@@ -22,19 +22,19 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
import org.apache.storm.messaging.netty.BackPressureStatus;
+import org.apache.storm.shade.org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.storm.shade.org.apache.commons.lang.builder.ToStringStyle;
import org.apache.storm.utils.JCQueue;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import java.util.stream.Collectors;
-import org.apache.storm.shade.org.apache.commons.lang.builder.ToStringBuilder;
-import org.apache.storm.shade.org.apache.commons.lang.builder.ToStringStyle;
-
-/***
- * Tracks the BackPressure status.
+/**
+ * Tracks the BackPressure status.
*/
public class BackPressureTracker {
static final Logger LOG = LoggerFactory.getLogger(BackPressureTracker.class);
@@ -53,9 +53,11 @@
tasks.get(taskId).backpressure.set(false);
}
- /***
+ /**
* Record BP for a task.
- * This is called by transferLocalBatch() on NettyWorker thread
+ *
+ * <p>This is called by transferLocalBatch() on NettyWorker thread
+ *
* @return true if an update was recorded, false if taskId is already under BP
*/
public boolean recordBackPressure(Integer taskId) {
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
index 175a91a..23e577e 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
@@ -261,17 +261,16 @@
});
workerState.checkForUpdatedBlobsTimer.scheduleRecurring(0,
- (Integer) conf
- .getOrDefault(Config.WORKER_BLOB_UPDATE_POLL_INTERVAL_SECS, 10),
- () -> {
- try {
- LOG.debug("Checking if blobs have updated");
- updateBlobUpdates();
- } catch (IOException e) {
- // IOException from reading the version files to be ignored
- LOG.error(e.getStackTrace().toString());
- }
- }
+ (Integer) conf.getOrDefault(Config.WORKER_BLOB_UPDATE_POLL_INTERVAL_SECS, 10),
+ () -> {
+ try {
+ LOG.debug("Checking if blobs have updated");
+ updateBlobUpdates();
+ } catch (IOException e) {
+ // IOException from reading the version files to be ignored
+ LOG.error(e.getStackTrace().toString());
+ }
+ }
);
// The jitter allows the clients to get the data at different times, and avoids thundering herd
@@ -309,15 +308,15 @@
}
workerState.flushTupleTimer.scheduleRecurringMs(flushIntervalMillis, flushIntervalMillis,
- () -> {
- // send flush tuple to all local executors
- for (int i = 0; i < executors.size(); i++) {
- IRunningExecutor exec = executors.get(i);
- if (exec.getExecutorId().get(0) != Constants.SYSTEM_TASK_ID) {
- exec.publishFlushTuple();
- }
- }
- }
+ () -> {
+ // send flush tuple to all local executors
+ for (int i = 0; i < executors.size(); i++) {
+ IRunningExecutor exec = executors.get(i);
+ if (exec.getExecutorId().get(0) != Constants.SYSTEM_TASK_ID) {
+ exec.publishFlushTuple();
+ }
+ }
+ }
);
LOG.info("Flush tuple will be generated every {} millis", flushIntervalMillis);
}
@@ -367,11 +366,11 @@
.toMap(IRunningExecutor::getExecutorId,
IRunningExecutor::renderStats)));
}
- Map<String, Object> zkHB = ClientStatsUtil.mkZkWorkerHb(workerState.topologyId, stats, workerState.uptime.upTime());
+ Map<String, Object> zkHb = ClientStatsUtil.mkZkWorkerHb(workerState.topologyId, stats, workerState.uptime.upTime());
try {
workerState.stormClusterState
.workerHeartbeat(workerState.topologyId, workerState.assignmentId, (long) workerState.port,
- ClientStatsUtil.thriftifyZkWorkerHb(zkHB));
+ ClientStatsUtil.thriftifyZkWorkerHb(zkHb));
} catch (Exception ex) {
LOG.error("Worker failed to write heartbeats to ZK or Pacemaker...will retry", ex);
}
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
index 913261d..f380769 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
@@ -152,11 +152,19 @@
private final Collection<IAutoCredentials> autoCredentials;
private final StormMetricRegistry metricRegistry;
- public WorkerState(Map<String, Object> conf, IContext mqContext, String topologyId, String assignmentId,
- Supplier<SupervisorIfaceFactory> supervisorIfaceSupplier, int port, String workerId, Map<String, Object> topologyConf, IStateStorage stateStorage,
- IStormClusterState stormClusterState, Collection<IAutoCredentials> autoCredentials,
- StormMetricRegistry metricRegistry) throws IOException,
- InvalidTopologyException {
+ public WorkerState(Map<String, Object> conf,
+ IContext mqContext,
+ String topologyId,
+ String assignmentId,
+ Supplier<SupervisorIfaceFactory> supervisorIfaceSupplier,
+ int port,
+ String workerId,
+ Map<String, Object> topologyConf,
+ IStateStorage stateStorage,
+ IStormClusterState stormClusterState,
+ Collection<IAutoCredentials> autoCredentials,
+ StormMetricRegistry metricRegistry) throws IOException,
+ InvalidTopologyException {
this.metricRegistry = metricRegistry;
this.autoCredentials = autoCredentials;
this.conf = conf;
@@ -230,9 +238,9 @@
this.receiver = this.mqContext.bind(topologyId, port, cb, newConnectionResponse);
}
- private static double getQueueLoad(JCQueue q) {
- JCQueue.QueueMetrics qMetrics = q.getMetrics();
- return ((double) qMetrics.population()) / qMetrics.capacity();
+ private static double getQueueLoad(JCQueue queue) {
+ JCQueue.QueueMetrics queueMetrics = queue.getMetrics();
+ return ((double) queueMetrics.population()) / queueMetrics.capacity();
}
public static boolean isConnectionReady(IConnection connection) {
@@ -463,7 +471,6 @@
public void refreshLoad(List<IRunningExecutor> execs) {
Set<Integer> remoteTasks = Sets.difference(new HashSet<>(outboundTasks), new HashSet<>(localTaskIds));
- Long now = System.currentTimeMillis();
Map<Integer, Double> localLoad = new HashMap<>();
for (IRunningExecutor exec : execs) {
double receiveLoad = getQueueLoad(exec.getReceiveQueue());
@@ -475,6 +482,7 @@
loadMapping.setLocal(localLoad);
loadMapping.setRemote(remoteLoad);
+ Long now = System.currentTimeMillis();
if (now > nextLoadUpdate.get()) {
receiver.sendLoadMetrics(localLoad);
nextLoadUpdate.set(now + LOAD_REFRESH_INTERVAL_MS);
@@ -498,14 +506,14 @@
int delaySecs = 0;
int recurSecs = 1;
refreshActiveTimer.schedule(delaySecs,
- () -> {
- if (areAllConnectionsReady()) {
- LOG.info("All connections are ready for worker {}:{} with id {}", assignmentId, port, workerId);
- isWorkerActive.countDown();
- } else {
- refreshActiveTimer.schedule(recurSecs, () -> activateWorkerWhenAllConnectionsReady(), false, 0);
- }
- }
+ () -> {
+ if (areAllConnectionsReady()) {
+ LOG.info("All connections are ready for worker {}:{} with id {}", assignmentId, port, workerId);
+ isWorkerActive.countDown();
+ } else {
+ refreshActiveTimer.schedule(recurSecs, () -> activateWorkerWhenAllConnectionsReady(), false, 0);
+ }
+ }
);
}
@@ -649,7 +657,7 @@
try (SupervisorIfaceFactory fac = supervisorIfaceSupplier.get()) {
return fac.getIface().getLocalAssignmentForStorm(topologyId);
} catch (Throwable e) {
- //if any error/exception thrown, fetch it from zookeeper
+ //if any error/exception thrown, fetch it from zookeeper
Assignment assignment = stormClusterState.remoteAssignmentInfo(topologyId, null);
if (assignment == null) {
throw new RuntimeException("Failed to read worker assignment."
@@ -666,8 +674,8 @@
if (recvBatchSize > recvQueueSize / 2) {
throw new IllegalArgumentException(Config.TOPOLOGY_PRODUCER_BATCH_SIZE + ":" + recvBatchSize
- + " is greater than half of " + Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE + ":" +
- recvQueueSize);
+ + " is greater than half of " + Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE + ":"
+ + recvQueueSize);
}
IWaitStrategy backPressureWaitStrategy = IWaitStrategy.createBackPressureWaitStrategy(topologyConf);
@@ -704,6 +712,7 @@
}
/**
+ * Get worker outbound tasks.
* @return seq of task ids that receive messages from this worker
*/
private Set<Integer> workerOutboundTasks() {
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java b/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
index 738af9b..a83e6ca 100644
--- a/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyBlobStoreUtils.java
@@ -30,6 +30,7 @@
return BLOB_DEPENDENCIES_PREFIX + key;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static String applyUUIDToFileName(String fileName) {
String fileNameWithExt = Files.getNameWithoutExtension(fileName);
String ext = Files.getFileExtension(fileName);
diff --git a/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
index 1d74d88..1f6b4f4 100644
--- a/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
@@ -37,11 +37,11 @@
import org.apache.storm.generated.KeyNotFoundException;
import org.apache.storm.generated.SettableBlobMeta;
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
+import org.apache.storm.shade.org.apache.commons.io.IOUtils;
import org.apache.storm.utils.ObjectReader;
import org.apache.storm.utils.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.storm.shade.org.apache.commons.io.IOUtils;
public class DependencyUploader {
public static final Logger LOG = LoggerFactory.getLogger(DependencyUploader.class);
@@ -163,7 +163,7 @@
AtomicOutputStream blob = null;
try {
blob = getBlobStore().createBlob(key, new SettableBlobMeta(acls));
- try(InputStream in = Files.newInputStream(dependency.toPath())) {
+ try (InputStream in = Files.newInputStream(dependency.toPath())) {
IOUtils.copy(in, blob, this.uploadChuckSize);
}
blob.close();
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java b/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
index e388de7..c4ca44a 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/DRPCInvocationsClient.java
@@ -25,6 +25,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface {
public static final Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class);
private final AtomicReference<DistributedRPCInvocations.Client> client = new AtomicReference<>();
@@ -35,7 +36,7 @@
super(conf, ThriftConnectionType.DRPC_INVOCATIONS, host, port, null);
this.host = host;
this.port = port;
- client.set(new DistributedRPCInvocations.Client(_protocol));
+ client.set(new DistributedRPCInvocations.Client(protocol));
}
public String getHost() {
@@ -49,7 +50,7 @@
public void reconnectClient() throws TException {
if (client.get() == null) {
reconnect();
- client.set(new DistributedRPCInvocations.Client(_protocol));
+ client.set(new DistributedRPCInvocations.Client(protocol));
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
index f8b6bc7..8756fca 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
@@ -49,38 +49,39 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class DRPCSpout extends BaseRichSpout {
public static final Logger LOG = LoggerFactory.getLogger(DRPCSpout.class);
//ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
- static final long serialVersionUID = 2387848310969237877L;
- final String _function;
- final String _local_drpc_id;
- SpoutOutputCollector _collector;
- List<DRPCInvocationsClient> _clients = new ArrayList<>();
- transient LinkedList<Future<Void>> _futures = null;
- transient ExecutorService _backround = null;
+ private static final long serialVersionUID = 2387848310969237877L;
+ private final String function;
+ private final String localDrpcId;
+ private SpoutOutputCollector collector;
+ private List<DRPCInvocationsClient> clients = new ArrayList<>();
+ private transient LinkedList<Future<Void>> futures = null;
+ private transient ExecutorService background = null;
public DRPCSpout(String function) {
- _function = function;
+ this.function = function;
if (DRPCClient.isLocalOverride()) {
- _local_drpc_id = DRPCClient.getOverrideServiceId();
+ localDrpcId = DRPCClient.getOverrideServiceId();
} else {
- _local_drpc_id = null;
+ localDrpcId = null;
}
}
public DRPCSpout(String function, ILocalDRPC drpc) {
- _function = function;
- _local_drpc_id = drpc.getServiceId();
+ this.function = function;
+ localDrpcId = drpc.getServiceId();
}
public String get_function() {
- return _function;
+ return function;
}
private void reconnectAsync(final DRPCInvocationsClient client) {
- _futures.add(_backround.submit(new Callable<Void>() {
+ futures.add(background.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
client.reconnectClient();
@@ -99,7 +100,7 @@
}
private void checkFutures() {
- Iterator<Future<Void>> i = _futures.iterator();
+ Iterator<Future<Void>> i = futures.iterator();
while (i.hasNext()) {
Future<Void> f = i.next();
if (f.isDone()) {
@@ -115,12 +116,12 @@
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _collector = collector;
- if (_local_drpc_id == null) {
- _backround = new ExtendedThreadPoolExecutor(0, Integer.MAX_VALUE,
+ this.collector = collector;
+ if (localDrpcId == null) {
+ background = new ExtendedThreadPoolExecutor(0, Integer.MAX_VALUE,
60L, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>());
- _futures = new LinkedList<>();
+ futures = new LinkedList<>();
int numTasks = context.getComponentTasks(context.getThisComponentId()).size();
int index = context.getThisTaskIndex();
@@ -133,11 +134,11 @@
if (numTasks < servers.size()) {
for (String s : servers) {
- _futures.add(_backround.submit(new Adder(s, port, conf)));
+ futures.add(background.submit(new Adder(s, port, conf)));
}
} else {
int i = index % servers.size();
- _futures.add(_backround.submit(new Adder(servers.get(i), port, conf)));
+ futures.add(background.submit(new Adder(servers.get(i), port, conf)));
}
}
@@ -145,22 +146,22 @@
@Override
public void close() {
- for (DRPCInvocationsClient client : _clients) {
+ for (DRPCInvocationsClient client : clients) {
client.close();
}
}
@Override
public void nextTuple() {
- if (_local_drpc_id == null) {
+ if (localDrpcId == null) {
int size = 0;
- synchronized (_clients) {
- size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end
+ synchronized (clients) {
+ size = clients.size(); //This will only ever grow, so no need to worry about falling off the end
}
for (int i = 0; i < size; i++) {
DRPCInvocationsClient client;
- synchronized (_clients) {
- client = _clients.get(i);
+ synchronized (clients) {
+ client = clients.get(i);
}
if (!client.isConnected()) {
LOG.warn("DRPCInvocationsClient [{}:{}] is not connected.", client.getHost(), client.getPort());
@@ -168,13 +169,13 @@
continue;
}
try {
- DRPCRequest req = client.fetchRequest(_function);
+ DRPCRequest req = client.fetchRequest(function);
if (req.get_request_id().length() > 0) {
Map<String, Object> returnInfo = new HashMap<>();
returnInfo.put("id", req.get_request_id());
returnInfo.put("host", client.getHost());
returnInfo.put("port", client.getPort());
- _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)),
+ collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)),
new DRPCMessageId(req.get_request_id(), i));
break;
}
@@ -190,16 +191,16 @@
}
checkFutures();
} else {
- DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id);
+ DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(localDrpcId);
if (drpc != null) { // can happen during shutdown of drpc while topology is still up
try {
- DRPCRequest req = drpc.fetchRequest(_function);
+ DRPCRequest req = drpc.fetchRequest(function);
if (req.get_request_id().length() > 0) {
Map<String, Object> returnInfo = new HashMap<>();
returnInfo.put("id", req.get_request_id());
- returnInfo.put("host", _local_drpc_id);
+ returnInfo.put("host", localDrpcId);
returnInfo.put("port", 0);
- _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)),
+ collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)),
new DRPCMessageId(req.get_request_id(), 0));
}
} catch (AuthorizationException aze) {
@@ -220,10 +221,10 @@
DRPCMessageId did = (DRPCMessageId) msgId;
DistributedRPCInvocations.Iface client;
- if (_local_drpc_id == null) {
- client = _clients.get(did.index);
+ if (localDrpcId == null) {
+ client = clients.get(did.index);
} else {
- client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id);
+ client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(localDrpcId);
}
int retryCnt = 0;
@@ -252,6 +253,7 @@
declarer.declare(new Fields("args", "return-info"));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static class DRPCMessageId {
String id;
int index;
@@ -276,8 +278,8 @@
@Override
public Void call() throws Exception {
DRPCInvocationsClient c = new DRPCInvocationsClient(conf, server, port);
- synchronized (_clients) {
- _clients.add(c);
+ synchronized (clients) {
+ clients.add(c);
}
return null;
}
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java b/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
index 543e22a..d4b7214 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/JoinResult.java
@@ -30,10 +30,10 @@
public class JoinResult extends BaseRichBolt {
public static final Logger LOG = LoggerFactory.getLogger(JoinResult.class);
- String returnComponent;
- Map<Object, Tuple> returns = new HashMap<>();
- Map<Object, Tuple> results = new HashMap<>();
- OutputCollector _collector;
+ private String returnComponent;
+ private Map<Object, Tuple> returns = new HashMap<>();
+ private Map<Object, Tuple> results = new HashMap<>();
+ private OutputCollector collector;
public JoinResult(String returnComponent) {
this.returnComponent = returnComponent;
@@ -41,7 +41,7 @@
@Override
public void prepare(Map<String, Object> map, TopologyContext context, OutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
@@ -60,9 +60,9 @@
List<Tuple> anchors = new ArrayList<>();
anchors.add(result);
anchors.add(returner);
- _collector.emit(anchors, new Values("" + result.getValue(1), returner.getValue(1)));
- _collector.ack(result);
- _collector.ack(returner);
+ collector.emit(anchors, new Values("" + result.getValue(1), returner.getValue(1)));
+ collector.ack(result);
+ collector.ack(returner);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java b/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
index fc10f86..7a358bf 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/KeyedFairBolt.java
@@ -26,13 +26,13 @@
public class KeyedFairBolt implements IRichBolt, FinishedCallback {
- IRichBolt _delegate;
- KeyedRoundRobinQueue<Tuple> _rrQueue;
- Thread _executor;
- FinishedCallback _callback;
+ IRichBolt delegate;
+ KeyedRoundRobinQueue<Tuple> rrQueue;
+ Thread executor;
+ FinishedCallback callback;
public KeyedFairBolt(IRichBolt delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
public KeyedFairBolt(IBasicBolt delegate) {
@@ -41,48 +41,48 @@
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- if (_delegate instanceof FinishedCallback) {
- _callback = (FinishedCallback) _delegate;
+ if (delegate instanceof FinishedCallback) {
+ callback = (FinishedCallback) delegate;
}
- _delegate.prepare(topoConf, context, collector);
- _rrQueue = new KeyedRoundRobinQueue<Tuple>();
- _executor = new Thread(new Runnable() {
+ delegate.prepare(topoConf, context, collector);
+ rrQueue = new KeyedRoundRobinQueue<Tuple>();
+ executor = new Thread(new Runnable() {
@Override
public void run() {
try {
while (true) {
- _delegate.execute(_rrQueue.take());
+ delegate.execute(rrQueue.take());
}
} catch (InterruptedException e) {
-
+ //ignore
}
}
});
- _executor.setDaemon(true);
- _executor.start();
+ executor.setDaemon(true);
+ executor.start();
}
@Override
public void execute(Tuple input) {
Object key = input.getValue(0);
- _rrQueue.add(key, input);
+ rrQueue.add(key, input);
}
@Override
public void cleanup() {
- _executor.interrupt();
- _delegate.cleanup();
+ executor.interrupt();
+ delegate.cleanup();
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _delegate.declareOutputFields(declarer);
+ delegate.declareOutputFields(declarer);
}
@Override
public void finishedId(Object id) {
- if (_callback != null) {
- _callback.finishedId(id);
+ if (callback != null) {
+ callback.finishedId(id);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
index 5ef3549..d026ec2 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCInputDeclarer.java
@@ -16,6 +16,7 @@
import org.apache.storm.topology.ComponentConfigurationDeclarer;
import org.apache.storm.tuple.Fields;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface LinearDRPCInputDeclarer extends ComponentConfigurationDeclarer<LinearDRPCInputDeclarer> {
public LinearDRPCInputDeclarer fieldsGrouping(Fields fields);
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
index 53f8eb9..6512976 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/LinearDRPCTopologyBuilder.java
@@ -41,7 +41,7 @@
import org.apache.storm.topology.TopologyBuilder;
import org.apache.storm.tuple.Fields;
-
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class LinearDRPCTopologyBuilder {
String function;
List<Component> components = new ArrayList<>();
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java b/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
index b120b51..6f40645 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/ReturnResults.java
@@ -37,15 +37,15 @@
public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class);
//ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
static final long serialVersionUID = -774882142710631591L;
- OutputCollector _collector;
+ OutputCollector collector;
boolean local;
- Map<String, Object> _conf;
- Map<List, DRPCInvocationsClient> _clients = new HashMap<List, DRPCInvocationsClient>();
+ Map<String, Object> conf;
+ Map<List, DRPCInvocationsClient> clients = new HashMap<List, DRPCInvocationsClient>();
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _conf = topoConf;
- _collector = collector;
+ conf = topoConf;
+ this.collector = collector;
local = topoConf.get(Config.STORM_CLUSTER_MODE).equals("local");
}
@@ -59,7 +59,7 @@
retMap = (Map<String, Object>) JSONValue.parseWithException(returnInfo);
} catch (ParseException e) {
LOG.error("Parseing returnInfo failed", e);
- _collector.fail(input);
+ collector.fail(input);
return;
}
final String host = (String) retMap.get("host");
@@ -69,19 +69,21 @@
if (local) {
client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host);
} else {
- List server = new ArrayList() {{
- add(host);
- add(port);
- }};
+ List server = new ArrayList() {
+ {
+ add(host);
+ add(port);
+ }
+ };
- if (!_clients.containsKey(server)) {
+ if (!clients.containsKey(server)) {
try {
- _clients.put(server, new DRPCInvocationsClient(_conf, host, port));
+ clients.put(server, new DRPCInvocationsClient(conf, host, port));
} catch (TTransportException ex) {
throw new RuntimeException(ex);
}
}
- client = _clients.get(server);
+ client = clients.get(server);
}
@@ -91,16 +93,16 @@
retryCnt++;
try {
client.result(id, result);
- _collector.ack(input);
+ collector.ack(input);
break;
} catch (AuthorizationException aze) {
LOG.error("Not authorized to return results to DRPC server", aze);
- _collector.fail(input);
+ collector.fail(input);
throw new RuntimeException(aze);
} catch (TException tex) {
if (retryCnt >= maxRetries) {
LOG.error("Failed to return results to DRPC server", tex);
- _collector.fail(input);
+ collector.fail(input);
}
reconnectClient((DRPCInvocationsClient) client);
}
@@ -121,7 +123,7 @@
@Override
public void cleanup() {
- for (DRPCInvocationsClient c : _clients.values()) {
+ for (DRPCInvocationsClient c : clients.values()) {
c.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index c69596e..7e687ad 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -325,20 +325,20 @@
for (final Integer interval : intervalToTaskToMetricToRegistry.keySet()) {
StormTimer timerTask = workerData.getUserTimer();
timerTask.scheduleRecurring(interval, interval,
- () -> {
- TupleImpl tuple =
- new TupleImpl(workerTopologyContext, new Values(interval), Constants.SYSTEM_COMPONENT_ID,
- (int) Constants.SYSTEM_TASK_ID, Constants.METRICS_TICK_STREAM_ID);
- AddressedTuple metricsTickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple);
- try {
- receiveQueue.publish(metricsTickTuple);
- receiveQueue.flush(); // avoid buffering
- } catch (InterruptedException e) {
- LOG.warn("Thread interrupted when publishing metrics. Setting interrupt flag.");
- Thread.currentThread().interrupt();
- return;
- }
- }
+ () -> {
+ TupleImpl tuple =
+ new TupleImpl(workerTopologyContext, new Values(interval), Constants.SYSTEM_COMPONENT_ID,
+ (int) Constants.SYSTEM_TASK_ID, Constants.METRICS_TICK_STREAM_ID);
+ AddressedTuple metricsTickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple);
+ try {
+ receiveQueue.publish(metricsTickTuple);
+ receiveQueue.flush(); // avoid buffering
+ } catch (InterruptedException e) {
+ LOG.warn("Thread interrupted when publishing metrics. Setting interrupt flag.");
+ Thread.currentThread().interrupt();
+ return;
+ }
+ }
);
}
}
@@ -355,21 +355,21 @@
} else {
StormTimer timerTask = workerData.getUserTimer();
timerTask.scheduleRecurring(tickTimeSecs, tickTimeSecs,
- () -> {
- TupleImpl tuple = new TupleImpl(workerTopologyContext, new Values(tickTimeSecs),
- Constants.SYSTEM_COMPONENT_ID,
- (int) Constants.SYSTEM_TASK_ID,
- Constants.SYSTEM_TICK_STREAM_ID);
- AddressedTuple tickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple);
- try {
- receiveQueue.publish(tickTuple);
- receiveQueue.flush(); // avoid buffering
- } catch (InterruptedException e) {
- LOG.warn("Thread interrupted when emitting tick tuple. Setting interrupt flag.");
- Thread.currentThread().interrupt();
- return;
- }
- }
+ () -> {
+ TupleImpl tuple = new TupleImpl(workerTopologyContext, new Values(tickTimeSecs),
+ Constants.SYSTEM_COMPONENT_ID,
+ (int) Constants.SYSTEM_TASK_ID,
+ Constants.SYSTEM_TICK_STREAM_ID);
+ AddressedTuple tickTuple = new AddressedTuple(AddressedTuple.BROADCAST_DEST, tuple);
+ try {
+ receiveQueue.publish(tickTuple);
+ receiveQueue.flush(); // avoid buffering
+ } catch (InterruptedException e) {
+ LOG.warn("Thread interrupted when emitting tick tuple. Setting interrupt flag.");
+ Thread.currentThread().interrupt();
+ return;
+ }
+ }
);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java b/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
index eee553e..4121a85 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/ExecutorTransfer.java
@@ -36,7 +36,7 @@
private int indexingBase = 0;
private ArrayList<JCQueue> localReceiveQueues; // [taskId-indexingBase] => queue : List of all recvQs local to this worker
private AtomicReferenceArray<JCQueue> queuesToFlush;
- // [taskId-indexingBase] => queue, some entries can be null. : outbound Qs for this executor instance
+ // [taskId-indexingBase] => queue, some entries can be null. : outbound Qs for this executor instance
public ExecutorTransfer(WorkerState workerData, Map<String, Object> topoConf) {
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 2ac76ec..4fbf1b3 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -33,7 +33,7 @@
import org.apache.storm.messaging.IConnection;
import org.apache.storm.metric.api.IMetricsRegistrant;
import org.apache.storm.policy.IWaitStrategy;
-import org.apache.storm.policy.IWaitStrategy.WAIT_SITUATION;
+import org.apache.storm.policy.IWaitStrategy.WaitSituation;
import org.apache.storm.policy.WaitStrategyPark;
import org.apache.storm.security.auth.IAutoCredentials;
import org.apache.storm.shade.com.google.common.collect.ImmutableMap;
@@ -75,10 +75,10 @@
this.consumeWaitStrategy = makeSystemBoltWaitStrategy();
} else {
this.consumeWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_BOLT_WAIT_STRATEGY));
- this.consumeWaitStrategy.prepare(topoConf, WAIT_SITUATION.BOLT_WAIT);
+ this.consumeWaitStrategy.prepare(topoConf, WaitSituation.BOLT_WAIT);
}
this.backPressureWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_STRATEGY));
- this.backPressureWaitStrategy.prepare(topoConf, WAIT_SITUATION.BACK_PRESSURE_WAIT);
+ this.backPressureWaitStrategy.prepare(topoConf, WaitSituation.BACK_PRESSURE_WAIT);
this.stats = new BoltExecutorStats(ConfigUtils.samplingRate(this.getTopoConf()),
ObjectReader.getInt(this.getTopoConf().get(Config.NUM_STAT_BUCKETS)));
this.builtInMetrics = new BuiltinBoltMetrics(stats);
@@ -88,7 +88,7 @@
WaitStrategyPark ws = new WaitStrategyPark();
Map<String, Object> conf = new HashMap<>();
conf.put(Config.TOPOLOGY_BOLT_WAIT_PARK_MICROSEC, 5000);
- ws.prepare(conf, WAIT_SITUATION.BOLT_WAIT);
+ ws.prepare(conf, WaitSituation.BOLT_WAIT);
return ws;
}
@@ -239,8 +239,8 @@
LOG.info("Execute done TUPLE {} TASK: {} DELTA: {}", tuple, taskId, delta);
}
TopologyContext topologyContext = idToTask.get(taskId - idToTaskBase).getUserContext();
- if (!topologyContext.getHooks().isEmpty()) // perf critical check to avoid unnecessary allocation
- {
+ if (!topologyContext.getHooks().isEmpty()) {
+ // perf critical check to avoid unnecessary allocation
new BoltExecuteInfo(tuple, taskId, delta).applyOn(topologyContext);
}
if (delta >= 0) {
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
index 5ea7838..da9af50 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltOutputCollectorImpl.java
@@ -98,8 +98,8 @@
if (rootIds.size() > 0) {
long edgeId = MessageId.generateId(random);
((TupleImpl) a).updateAckVal(edgeId);
- for (Long root_id : rootIds) {
- putXor(anchorsToIds, root_id, edgeId);
+ for (Long rootId : rootIds) {
+ putXor(anchorsToIds, rootId, edgeId);
}
}
}
@@ -202,7 +202,7 @@
private void putXor(Map<Long, Long> pending, Long key, Long id) {
Long curr = pending.get(key);
if (curr == null) {
- curr = 0l;
+ curr = 0L;
}
pending.put(key, Utils.bitXor(curr, id));
}
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
index 0c45d56..c10ab2e 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
@@ -33,7 +33,7 @@
import org.apache.storm.hooks.info.SpoutAckInfo;
import org.apache.storm.hooks.info.SpoutFailInfo;
import org.apache.storm.policy.IWaitStrategy;
-import org.apache.storm.policy.IWaitStrategy.WAIT_SITUATION;
+import org.apache.storm.policy.IWaitStrategy.WaitSituation;
import org.apache.storm.shade.com.google.common.collect.ImmutableMap;
import org.apache.storm.spout.ISpout;
import org.apache.storm.spout.SpoutOutputCollector;
@@ -75,9 +75,9 @@
public SpoutExecutor(final WorkerState workerData, final List<Long> executorId, Map<String, String> credentials) {
super(workerData, executorId, credentials, ClientStatsUtil.SPOUT);
this.spoutWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_SPOUT_WAIT_STRATEGY));
- this.spoutWaitStrategy.prepare(topoConf, WAIT_SITUATION.SPOUT_WAIT);
+ this.spoutWaitStrategy.prepare(topoConf, WaitSituation.SPOUT_WAIT);
this.backPressureWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_STRATEGY));
- this.backPressureWaitStrategy.prepare(topoConf, WAIT_SITUATION.BACK_PRESSURE_WAIT);
+ this.backPressureWaitStrategy.prepare(topoConf, WaitSituation.BACK_PRESSURE_WAIT);
this.lastActive = new AtomicBoolean(false);
this.hasAckers = StormCommon.hasAckers(topoConf);
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
index b71b331..c370d6a 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutOutputCollectorImpl.java
@@ -46,7 +46,7 @@
private final RotatingMap<Long, TupleInfo> pending;
private final long spoutExecutorThdId;
private TupleInfo globalTupleInfo = new TupleInfo();
- // thread safety: assumes Collector.emit*() calls are externally synchronized (if needed).
+ // thread safety: assumes Collector.emit*() calls are externally synchronized (if needed).
@SuppressWarnings("unused")
public SpoutOutputCollectorImpl(ISpout spout, SpoutExecutor executor, Task taskData,
@@ -144,7 +144,6 @@
}
if (needAck) {
- boolean sample = executor.samplerCheck();
TupleInfo info = new TupleInfo();
info.setTaskId(this.taskId);
info.setStream(stream);
@@ -153,6 +152,7 @@
if (isDebug) {
info.setValues(values);
}
+ boolean sample = executor.samplerCheck();
if (sample) {
info.setTimestamp(System.currentTimeMillis());
}
@@ -164,8 +164,8 @@
// Reusing TupleInfo object as we directly call executor.ackSpoutMsg() & are not sending msgs. perf critical
if (isDebug) {
if (spoutExecutorThdId != Thread.currentThread().getId()) {
- throw new RuntimeException("Detected background thread emitting tuples for the spout. " +
- "Spout Output Collector should only emit from the main spout executor thread.");
+ throw new RuntimeException("Detected background thread emitting tuples for the spout. "
+ + "Spout Output Collector should only emit from the main spout executor thread.");
}
}
globalTupleInfo.clear();
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
index eb532f7..9630b21 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/CustomStreamGrouping.java
@@ -23,7 +23,7 @@
* Tells the stream grouping at runtime the tasks in the target bolt. This information should be used in chooseTasks to determine the
* target tasks.
*
- * It also tells the grouping the metadata on the stream this grouping will be used on.
+ * <p>It also tells the grouping the metadata on the stream this grouping will be used on.
*/
void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks);
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/Load.java b/storm-client/src/jvm/org/apache/storm/grouping/Load.java
index 2c6662d..a809267 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/Load.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/Load.java
@@ -21,7 +21,7 @@
private double connectionLoad = 0.0; //0 no load to 1 fully loaded
/**
- * Create a new load
+ * Create a new load.
*
* @param hasMetrics have metrics been reported yet?
* @param boltLoad the load as reported by the bolt 0.0 no load 1.0 fully loaded
@@ -34,6 +34,7 @@
}
/**
+ * Check whether has metrics.
* @return true if metrics have been reported so far.
*/
public boolean hasMetrics() {
@@ -41,6 +42,7 @@
}
/**
+ * Get bolt load.
* @return the load as reported by the bolt.
*/
public double getBoltLoad() {
@@ -48,6 +50,7 @@
}
/**
+ * Get connection load.
* @return the load as reported by the connection
*/
public double getConnectionLoad() {
@@ -55,6 +58,7 @@
}
/**
+ * Get load.
* @return the load that is a combination of sub loads.
*/
public double getLoad() {
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java b/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
index 40fcd99..569d1c4 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/LoadMapping.java
@@ -17,12 +17,12 @@
import java.util.concurrent.atomic.AtomicReference;
/**
- * Holds a list of the current loads
+ * Holds a list of the current loads.
*/
public class LoadMapping {
private static final Load NOT_CONNECTED = new Load(false, 1.0, 1.0);
- private final AtomicReference<Map<Integer, Load>> _local = new AtomicReference<Map<Integer, Load>>(new HashMap<Integer, Load>());
- private final AtomicReference<Map<Integer, Load>> _remote = new AtomicReference<Map<Integer, Load>>(new HashMap<Integer, Load>());
+ private final AtomicReference<Map<Integer, Load>> local = new AtomicReference<Map<Integer, Load>>(new HashMap<Integer, Load>());
+ private final AtomicReference<Map<Integer, Load>> remote = new AtomicReference<Map<Integer, Load>>(new HashMap<Integer, Load>());
public void setLocal(Map<Integer, Double> local) {
Map<Integer, Load> newLocal = new HashMap<Integer, Load>();
@@ -31,21 +31,21 @@
newLocal.put(entry.getKey(), new Load(true, entry.getValue(), 0.0));
}
}
- _local.set(newLocal);
+ this.local.set(newLocal);
}
public void setRemote(Map<Integer, Load> remote) {
if (remote != null) {
- _remote.set(new HashMap<Integer, Load>(remote));
+ this.remote.set(new HashMap<Integer, Load>(remote));
} else {
- _remote.set(new HashMap<Integer, Load>());
+ this.remote.set(new HashMap<Integer, Load>());
}
}
public Load getLoad(int task) {
- Load ret = _local.get().get(task);
+ Load ret = local.get().get(task);
if (ret == null) {
- ret = _remote.get().get(task);
+ ret = remote.get().get(task);
}
if (ret == null) {
ret = NOT_CONNECTED;
diff --git a/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java b/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
index bba635e..d61494c 100644
--- a/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/grouping/PartialKeyGrouping.java
@@ -25,14 +25,14 @@
import org.apache.storm.tuple.Fields;
/**
- * A variation on FieldGrouping. This grouping operates on a partitioning of the incoming tuples (like a FieldGrouping), but it can send
- * Tuples from a given partition to multiple downstream tasks.
+ * A variation on FieldGrouping. This grouping operates on a partitioning of the incoming tuples (like a FieldGrouping),
+ * but it can send Tuples from a given partition to multiple downstream tasks.
*
- * Given a total pool of target tasks, this grouping will always send Tuples with a given key to one member of a subset of those tasks. Each
- * key is assigned a subset of tasks. Each tuple is then sent to one task from that subset.
+ * <p>Given a total pool of target tasks, this grouping will always send Tuples with a given key to one member of a
+ * subset of those tasks. Each key is assigned a subset of tasks. Each tuple is then sent to one task from that subset.
*
- * Notes: - the default TaskSelector ensures each task gets as close to a balanced number of Tuples as possible - the default
- * AssignmentCreator hashes the key and produces an assignment of two tasks
+ * <p>Notes: - the default TaskSelector ensures each task gets as close to a balanced number of Tuples as possible - the
+ * default AssignmentCreator hashes the key and produces an assignment of two tasks
*/
public class PartialKeyGrouping implements CustomStreamGrouping, Serializable {
private static final long serialVersionUID = -1672360572274911808L;
@@ -133,8 +133,8 @@
/**
* This interface is responsible for choosing a subset of the target tasks to use for a given key.
*
- * NOTE: whatever scheme you use to create the assignment should be deterministic. This may be executed on multiple Storm Workers, thus
- * each of them needs to come up with the same assignment for a given key.
+ * <p>NOTE: whatever scheme you use to create the assignment should be deterministic. This may be executed on
+ * multiple Storm Workers, thus each of them needs to come up with the same assignment for a given key.
*/
public interface AssignmentCreator extends Serializable {
int[] createAssignment(List<Integer> targetTasks, byte[] key);
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java b/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
index 2915f5c..0539ab0 100644
--- a/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
+++ b/storm-client/src/jvm/org/apache/storm/hooks/BaseWorkerHook.java
@@ -17,14 +17,14 @@
import org.apache.storm.task.WorkerTopologyContext;
/**
- * A BaseWorkerHook is a noop implementation of IWorkerHook. You may extends this class and implement any and/or all methods you need for
- * your workers.
+ * A BaseWorkerHook is a noop implementation of IWorkerHook. You may extends this class and implement any and/or all
+ * methods you need for your workers.
*/
public class BaseWorkerHook implements IWorkerHook, Serializable {
private static final long serialVersionUID = 2589466485198339529L;
/**
- * This method is called when a worker is started
+ * This method is called when a worker is started.
*
* @param topoConf The Storm configuration for this worker
* @param context This object can be used to get information about this worker's place within the topology
@@ -35,7 +35,7 @@
}
/**
- * This method is called right before a worker shuts down
+ * This method is called right before a worker shuts down.
*/
@Override
public void shutdown() {
diff --git a/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java b/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
index a0ec059..fe68bcc 100644
--- a/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
+++ b/storm-client/src/jvm/org/apache/storm/hooks/IWorkerHook.java
@@ -22,7 +22,7 @@
*/
public interface IWorkerHook extends Serializable {
/**
- * This method is called when a worker is started
+ * This method is called when a worker is started.
*
* @param topoConf The Storm configuration for this worker
* @param context This object can be used to get information about this worker's place within the topology
@@ -30,7 +30,7 @@
void start(Map<String, Object> topoConf, WorkerTopologyContext context);
/**
- * This method is called right before a worker shuts down
+ * This method is called right before a worker shuts down.
*/
void shutdown();
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java b/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
index 818a9a4..da20253 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/ConnectionWithStatus.java
@@ -15,7 +15,7 @@
public abstract class ConnectionWithStatus implements IConnection {
/**
- * whether this connection is available to transfer data
+ * whether this connection is available to transfer data.
*/
public abstract Status status();
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java b/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
index 8ab52fd..b038e02 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/DeserializingConnectionCallback.java
@@ -36,7 +36,7 @@
private final Map<String, Object> conf;
private final GeneralTopologyContext context;
- private final ThreadLocal<KryoTupleDeserializer> _des =
+ private final ThreadLocal<KryoTupleDeserializer> des =
new ThreadLocal<KryoTupleDeserializer>() {
@Override
protected KryoTupleDeserializer initialValue() {
@@ -60,7 +60,7 @@
@Override
public void recv(List<TaskMessage> batch) {
- KryoTupleDeserializer des = _des.get();
+ KryoTupleDeserializer des = this.des.get();
ArrayList<AddressedTuple> ret = new ArrayList<>(batch.size());
for (TaskMessage message : batch) {
Tuple tuple = des.deserialize(message.message());
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java b/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
index f713c7f..11e98ac 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IConnection.java
@@ -34,15 +34,13 @@
void sendBackPressureStatus(BackPressureStatus bpStatus);
/**
- * send batch messages
- *
- * @param msgs
+ * send batch messages.
*/
void send(Iterator<TaskMessage> msgs);
/**
- * Get the current load for the given tasks
+ * Get the current load for the given tasks.
*
* @param tasks the tasks to look for.
* @return a Load for each of the tasks it knows about.
@@ -50,14 +48,14 @@
Map<Integer, Load> getLoad(Collection<Integer> tasks);
/**
- * Get the port for this connection
+ * Get the port for this connection.
*
* @return The port this connection is using
*/
int getPort();
/**
- * close this connection
+ * close this connection.
*/
@Override
void close();
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java b/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
index d9b3f7b..5719eb8 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IConnectionCallback.java
@@ -19,7 +19,7 @@
*/
public interface IConnectionCallback {
/**
- * A batch of new messages have arrived to be processed
+ * A batch of new messages have arrived to be processed.
*
* @param batch the messages to be processed
*/
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/IContext.java b/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
index 057ae30..ac56a8a 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/IContext.java
@@ -19,44 +19,44 @@
/**
* This interface needs to be implemented for messaging plugin.
*
- * Messaging plugin is specified via Storm config parameter, storm.messaging.transport.
+ * <p>Messaging plugin is specified via Storm config parameter, storm.messaging.transport.
*
- * A messaging plugin should have a default constructor and implements IContext interface. Upon construction, we will invoke
+ * <p>A messaging plugin should have a default constructor and implements IContext interface. Upon construction, we will invoke
* IContext::prepare(topoConf) to enable context to be configured according to storm configuration.
*/
public interface IContext {
/**
- * This method is invoked at the startup of messaging plugin
+ * This method is invoked at the startup of messaging plugin.
*
* @param topoConf storm configuration
*/
void prepare(Map<String, Object> topoConf);
/**
- * This method is invoked when a worker is unloading a messaging plugin
+ * This method is invoked when a worker is unloading a messaging plugin.
*/
void term();
/**
- * This method establishes a server side connection
+ * This method establishes a server side connection.
*
- * @param storm_id topology ID
+ * @param stormId topology ID
* @param port port #
* @param cb The callback to deliver received messages to
* @param newConnectionResponse Supplier of the initial message to send to new client connections
* @return server side connection
*/
- IConnection bind(String storm_id, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse);
+ IConnection bind(String stormId, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse);
/**
* This method establish a client side connection to a remote server
- * implementation should return a new connection every call
+ * implementation should return a new connection every call.
*
- * @param storm_id topology ID
+ * @param stormId topology ID
* @param host remote host
* @param port remote port
* @param remoteBpStatus array of booleans reflecting Back Pressure status of remote tasks.
* @return client side connection
*/
- IConnection connect(String storm_id, String host, int port, AtomicBoolean[] remoteBpStatus);
+ IConnection connect(String stormId, String host, int port, AtomicBoolean[] remoteBpStatus);
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java b/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
index c32a93b..9c2975d 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/TaskMessage.java
@@ -15,26 +15,26 @@
import java.nio.ByteBuffer;
public class TaskMessage {
- private int _task;
- private byte[] _message;
+ private int task;
+ private byte[] message;
public TaskMessage(int task, byte[] message) {
- _task = task;
- _message = message;
+ this.task = task;
+ this.message = message;
}
public int task() {
- return _task;
+ return task;
}
public byte[] message() {
- return _message;
+ return message;
}
public ByteBuffer serialize() {
- ByteBuffer bb = ByteBuffer.allocate(_message.length + 2);
- bb.putShort((short) _task);
- bb.put(_message);
+ ByteBuffer bb = ByteBuffer.allocate(message.length + 2);
+ bb.putShort((short) task);
+ bb.put(message);
return bb;
}
@@ -42,9 +42,9 @@
if (packet == null) {
return;
}
- _task = packet.getShort();
- _message = new byte[packet.limit() - 2];
- packet.get(_message);
+ task = packet.getShort();
+ message = new byte[packet.limit() - 2];
+ packet.get(message);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java b/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
index 9d3ec72..cc48eca 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/TransportFactory.java
@@ -24,13 +24,13 @@
public static IContext makeContext(Map<String, Object> topoConf) {
//get factory class name
- String transport_plugin_klassName = (String) topoConf.get(Config.STORM_MESSAGING_TRANSPORT);
- LOG.info("Storm peer transport plugin:" + transport_plugin_klassName);
+ String transportPluginClassName = (String) topoConf.get(Config.STORM_MESSAGING_TRANSPORT);
+ LOG.info("Storm peer transport plugin:" + transportPluginClassName);
IContext transport;
try {
//create a factory class
- Class klass = Class.forName(transport_plugin_klassName);
+ Class klass = Class.forName(transportPluginClassName);
//obtain a context object
Object obj = klass.newInstance();
if (obj instanceof IContext) {
@@ -45,7 +45,7 @@
transport = (IContext) method.invoke(obj, topoConf);
}
} catch (Exception e) {
- throw new RuntimeException("Fail to construct messaging plugin from plugin " + transport_plugin_klassName, e);
+ throw new RuntimeException("Fail to construct messaging plugin from plugin " + transportPluginClassName, e);
}
return transport;
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java b/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
index 2737dfb..69552f8 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/local/Context.java
@@ -13,7 +13,6 @@
package org.apache.storm.messaging.local;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@@ -38,7 +37,7 @@
public class Context implements IContext {
private static final Logger LOG = LoggerFactory.getLogger(Context.class);
- private final ConcurrentHashMap<String, LocalServer> _registry = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap<String, LocalServer> registry = new ConcurrentHashMap<>();
private static String getNodeKey(String nodeId, int port) {
return nodeId + "-" + port;
@@ -47,7 +46,7 @@
private LocalServer createLocalServer(String nodeId, int port, IConnectionCallback cb) {
String key = getNodeKey(nodeId, port);
LocalServer ret = new LocalServer(port, cb);
- LocalServer existing = _registry.put(key, ret);
+ LocalServer existing = registry.put(key, ret);
if (existing != null) {
//Can happen if worker is restarted in the same topology, e.g. due to blob update
LOG.info("Replacing existing server for key {}", existing, ret, key);
@@ -61,13 +60,13 @@
}
@Override
- public IConnection bind(String storm_id, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse) {
- return createLocalServer(storm_id, port, cb);
+ public IConnection bind(String stormId, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse) {
+ return createLocalServer(stormId, port, cb);
}
@Override
- public IConnection connect(String storm_id, String host, int port, AtomicBoolean[] remoteBpStatus) {
- return new LocalClient(storm_id, port);
+ public IConnection connect(String stormId, String host, int port, AtomicBoolean[] remoteBpStatus) {
+ return new LocalClient(stormId, port);
}
@Override
@@ -76,13 +75,13 @@
}
private class LocalServer implements IConnection {
- final ConcurrentHashMap<Integer, Double> _load = new ConcurrentHashMap<>();
+ final ConcurrentHashMap<Integer, Double> load = new ConcurrentHashMap<>();
final int port;
- final IConnectionCallback _cb;
+ final IConnectionCallback cb;
public LocalServer(int port, IConnectionCallback cb) {
this.port = port;
- this._cb = cb;
+ this.cb = cb;
}
@Override
@@ -94,7 +93,7 @@
public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
Map<Integer, Load> ret = new HashMap<>();
for (Integer task : tasks) {
- Double found = _load.get(task);
+ Double found = load.get(task);
if (found != null) {
ret.put(task, new Load(true, found, 0));
}
@@ -104,7 +103,7 @@
@Override
public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
- _load.putAll(taskToLoad);
+ load.putAll(taskToLoad);
}
@Override
@@ -125,16 +124,16 @@
private class LocalClient implements IConnection {
//Messages sent before the server registered a callback
- private final LinkedBlockingQueue<TaskMessage> _pendingDueToUnregisteredServer;
- private final ScheduledExecutorService _pendingFlusher;
+ private final LinkedBlockingQueue<TaskMessage> pendingDueToUnregisteredServer;
+ private final ScheduledExecutorService pendingFlusher;
private final int port;
private final String registryKey;
public LocalClient(String stormId, int port) {
this.port = port;
this.registryKey = getNodeKey(stormId, port);
- _pendingDueToUnregisteredServer = new LinkedBlockingQueue<>();
- _pendingFlusher = Executors.newScheduledThreadPool(1, new ThreadFactory() {
+ pendingDueToUnregisteredServer = new LinkedBlockingQueue<>();
+ pendingFlusher = Executors.newScheduledThreadPool(1, new ThreadFactory() {
@Override
public Thread newThread(Runnable runnable) {
Thread thread = new Thread(runnable);
@@ -143,7 +142,7 @@
return thread;
}
});
- _pendingFlusher.scheduleAtFixedRate(new Runnable() {
+ pendingFlusher.scheduleAtFixedRate(new Runnable() {
@Override
public void run() {
try {
@@ -159,34 +158,34 @@
private void flushPending() {
//Can't cache server in client, server can change when workers restart.
- LocalServer server = _registry.get(registryKey);
- if (server != null && !_pendingDueToUnregisteredServer.isEmpty()) {
+ LocalServer server = registry.get(registryKey);
+ if (server != null && !pendingDueToUnregisteredServer.isEmpty()) {
ArrayList<TaskMessage> ret = new ArrayList<>();
- _pendingDueToUnregisteredServer.drainTo(ret);
- server._cb.recv(ret);
+ pendingDueToUnregisteredServer.drainTo(ret);
+ server.cb.recv(ret);
}
}
@Override
public void send(Iterator<TaskMessage> msgs) {
- LocalServer server = _registry.get(registryKey);
+ LocalServer server = registry.get(registryKey);
if (server != null) {
flushPending();
ArrayList<TaskMessage> ret = new ArrayList<>();
while (msgs.hasNext()) {
ret.add(msgs.next());
}
- server._cb.recv(ret);
+ server.cb.recv(ret);
} else {
while (msgs.hasNext()) {
- _pendingDueToUnregisteredServer.add(msgs.next());
+ pendingDueToUnregisteredServer.add(msgs.next());
}
}
}
@Override
public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
- LocalServer server = _registry.get(registryKey);
+ LocalServer server = registry.get(registryKey);
if (server != null) {
return server.getLoad(tasks);
}
@@ -195,7 +194,7 @@
@Override
public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
- LocalServer server = _registry.get(registryKey);
+ LocalServer server = registry.get(registryKey);
if (server != null) {
server.sendLoadMetrics(taskToLoad);
}
@@ -213,9 +212,9 @@
@Override
public void close() {
- _pendingFlusher.shutdown();
+ pendingFlusher.shutdown();
try {
- _pendingFlusher.awaitTermination(5, TimeUnit.SECONDS);
+ pendingFlusher.awaitTermination(5, TimeUnit.SECONDS);
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted while awaiting flusher shutdown", e);
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
index e76f9d5..eac0f22 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Client.java
@@ -27,18 +27,14 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Supplier;
import org.apache.storm.Config;
import org.apache.storm.grouping.Load;
import org.apache.storm.messaging.ConnectionWithStatus;
-import org.apache.storm.messaging.IConnectionCallback;
import org.apache.storm.messaging.TaskMessage;
import org.apache.storm.metric.api.IStatefulObject;
import org.apache.storm.policy.IWaitStrategy;
-import org.apache.storm.policy.IWaitStrategy.WAIT_SITUATION;
+import org.apache.storm.policy.IWaitStrategy.WaitSituation;
import org.apache.storm.policy.WaitStrategyProgressive;
-import org.apache.storm.serialization.KryoValuesDeserializer;
-import org.apache.storm.serialization.KryoValuesSerializer;
import org.apache.storm.shade.io.netty.bootstrap.Bootstrap;
import org.apache.storm.shade.io.netty.buffer.PooledByteBufAllocator;
import org.apache.storm.shade.io.netty.channel.Channel;
@@ -166,7 +162,7 @@
} else {
waitStrategy = ReflectionUtils.newInstance(clazz);
}
- waitStrategy.prepare(topoConf, WAIT_SITUATION.BACK_PRESSURE_WAIT);
+ waitStrategy.prepare(topoConf, WaitSituation.BACK_PRESSURE_WAIT);
}
/**
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
index ca46c4f..03feaf8 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Context.java
@@ -34,7 +34,7 @@
private HashedWheelTimer clientScheduleService;
/**
- * initialization per Storm configuration
+ * initialization per Storm configuration.
*/
@Override
public void prepare(Map<String, Object> topoConf) {
@@ -52,26 +52,26 @@
}
/**
- * establish a server with a binding port
+ * establish a server with a binding port.
*/
@Override
- public synchronized IConnection bind(String storm_id, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse) {
+ public synchronized IConnection bind(String stormId, int port, IConnectionCallback cb, Supplier<Object> newConnectionResponse) {
Server server = new Server(topoConf, port, cb, newConnectionResponse);
serverConnections.add(server);
return server;
}
/**
- * establish a connection to a remote server
+ * establish a connection to a remote server.
*/
@Override
- public IConnection connect(String storm_id, String host, int port, AtomicBoolean[] remoteBpStatus) {
+ public IConnection connect(String stormId, String host, int port, AtomicBoolean[] remoteBpStatus) {
return new Client(topoConf, remoteBpStatus, workerEventLoopGroup,
clientScheduleService, host, port);
}
/**
- * terminate this context
+ * terminate this context.
*/
@Override
public synchronized void term() {
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
index 3836faf..beb4515 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/ControlMessage.java
@@ -31,6 +31,7 @@
}
/**
+ * Create message.
* @param encoded status code
* @return a control message per an encoded status code
*/
@@ -44,11 +45,11 @@
}
public static ControlMessage read(byte[] serial) {
- ByteBuf cm_buffer = Unpooled.wrappedBuffer(serial);
- try{
- return mkMessage(cm_buffer.getShort(0));
+ ByteBuf cmBuffer = Unpooled.wrappedBuffer(serial);
+ try {
+ return mkMessage(cmBuffer.getShort(0));
} finally {
- cm_buffer.release();
+ cmBuffer.release();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
index 5edaf95..450dc69 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslClientHandler.java
@@ -24,22 +24,22 @@
private static final Logger LOG = LoggerFactory
.getLogger(KerberosSaslClientHandler.class);
- private final long start_time;
+ private final long startTime;
private final ISaslClient client;
/**
* Used for client or server's token to send or receive from each other.
*/
private final Map<String, Object> topoConf;
- private final String jaas_section;
+ private final String jaasSection;
private final String host;
- public KerberosSaslClientHandler(ISaslClient client, Map<String, Object> topoConf, String jaas_section, String host) throws
+ public KerberosSaslClientHandler(ISaslClient client, Map<String, Object> topoConf, String jaasSection, String host) throws
IOException {
this.client = client;
this.topoConf = topoConf;
- this.jaas_section = jaas_section;
+ this.jaasSection = jaasSection;
this.host = host;
- start_time = System.currentTimeMillis();
+ startTime = System.currentTimeMillis();
}
@Override
@@ -56,7 +56,7 @@
if (saslNettyClient == null) {
LOG.debug("Creating saslNettyClient now for channel: {}",
channel);
- saslNettyClient = new KerberosSaslNettyClient(topoConf, jaas_section, host);
+ saslNettyClient = new KerberosSaslNettyClient(topoConf, jaasSection, host);
channel.attr(KerberosSaslNettyClientState.KERBEROS_SASL_NETTY_CLIENT).set(saslNettyClient);
}
LOG.debug("Going to initiate Kerberos negotiations.");
@@ -71,7 +71,7 @@
@Override
public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception {
- LOG.debug("send/recv time (ms): {}", (System.currentTimeMillis() - start_time));
+ LOG.debug("send/recv time (ms): {}", (System.currentTimeMillis() - startTime));
// examine the response message from server
if (message instanceof ControlMessage) {
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
index 352c5b3..c4d15fc 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClient.java
@@ -45,22 +45,22 @@
*/
private SaslClient saslClient;
private Subject subject;
- private String jaas_section;
+ private String jaasSection;
/**
* Create a KerberosSaslNettyClient for authentication with servers.
*/
- public KerberosSaslNettyClient(Map<String, Object> topoConf, String jaas_section, String host) {
+ public KerberosSaslNettyClient(Map<String, Object> topoConf, String jaasSection, String host) {
LOG.debug("KerberosSaslNettyClient: Creating SASL {} client to authenticate to server ",
SaslUtils.KERBEROS);
LOG.info("Creating Kerberos Client.");
- Configuration login_conf;
+ Configuration loginConf;
try {
- login_conf = ClientAuthUtils.getConfiguration(topoConf);
+ loginConf = ClientAuthUtils.getConfiguration(topoConf);
} catch (Throwable t) {
- LOG.error("Failed to get login_conf: ", t);
+ LOG.error("Failed to get loginConf: ", t);
throw t;
}
LOG.debug("KerberosSaslNettyClient: authmethod {}", SaslUtils.KERBEROS);
@@ -69,12 +69,12 @@
subject = null;
try {
- LOG.debug("Setting Configuration to login_config: {}", login_conf);
+ LOG.debug("Setting Configuration to login_config: {}", loginConf);
//specify a configuration object to be used
- Configuration.setConfiguration(login_conf);
+ Configuration.setConfiguration(loginConf);
//now login
LOG.debug("Trying to login.");
- Login login = new Login(jaas_section, ch);
+ Login login = new Login(jaasSection, ch);
subject = login.getSubject();
LOG.debug("Got Subject: {}", subject.toString());
} catch (LoginException ex) {
@@ -85,15 +85,15 @@
//check the credential of our principal
if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
LOG.error("Failed to verify user principal.");
- throw new RuntimeException("Fail to verify user principal with section \"" +
- jaas_section +
- "\" in login configuration file " +
- login_conf);
+ throw new RuntimeException("Fail to verify user principal with section \""
+ + jaasSection
+ + "\" in login configuration file "
+ + loginConf);
}
String serviceName = null;
try {
- serviceName = ClientAuthUtils.get(login_conf, jaas_section, "serviceName");
+ serviceName = ClientAuthUtils.get(loginConf, jaasSection, "serviceName");
} catch (IOException e) {
LOG.error("Failed to get service name.", e);
throw new RuntimeException(e);
@@ -176,7 +176,6 @@
* Implementation used to respond to SASL tokens from server.
*
* @param callbacks objects that indicate what credential information the server's SaslServer requires from the client.
- * @throws UnsupportedCallbackException
*/
@Override
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
index 29746ff..6c923b5 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyClientState.java
@@ -16,6 +16,7 @@
final class KerberosSaslNettyClientState {
- public static final AttributeKey<KerberosSaslNettyClient> KERBEROS_SASL_NETTY_CLIENT = AttributeKey.valueOf("kerberos.sasl.netty.client");
+ public static final AttributeKey<KerberosSaslNettyClient> KERBEROS_SASL_NETTY_CLIENT =
+ AttributeKey.valueOf("kerberos.sasl.netty.client");
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
index 466ee6d..e7f0c14 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServer.java
@@ -47,14 +47,14 @@
private Subject subject;
private List<String> authorizedUsers;
- KerberosSaslNettyServer(Map<String, Object> topoConf, String jaas_section, List<String> authorizedUsers) {
+ KerberosSaslNettyServer(Map<String, Object> topoConf, String jaasSection, List<String> authorizedUsers) {
this.authorizedUsers = authorizedUsers;
LOG.debug("Getting Configuration.");
- Configuration login_conf;
+ Configuration loginConf;
try {
- login_conf = ClientAuthUtils.getConfiguration(topoConf);
+ loginConf = ClientAuthUtils.getConfiguration(topoConf);
} catch (Throwable t) {
- LOG.error("Failed to get login_conf: ", t);
+ LOG.error("Failed to get loginConf: ", t);
throw t;
}
@@ -65,12 +65,12 @@
//login our principal
subject = null;
try {
- LOG.debug("Setting Configuration to login_config: {}", login_conf);
+ LOG.debug("Setting Configuration to login_config: {}", loginConf);
//specify a configuration object to be used
- Configuration.setConfiguration(login_conf);
+ Configuration.setConfiguration(loginConf);
//now login
LOG.debug("Trying to login.");
- Login login = new Login(jaas_section, ch);
+ Login login = new Login(jaasSection, ch);
subject = login.getSubject();
LOG.debug("Got Subject: {}", subject.toString());
} catch (LoginException ex) {
@@ -82,19 +82,19 @@
if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
LOG.error("Failed to verifyuser principal.");
throw new RuntimeException("Fail to verify user principal with section \""
- + jaas_section
+ + jaasSection
+ "\" in login configuration file "
- + login_conf);
+ + loginConf);
}
try {
LOG.info("Creating Kerberos Server.");
final CallbackHandler fch = ch;
Principal p = (Principal) subject.getPrincipals().toArray()[0];
- KerberosName kName = new KerberosName(p.getName());
- final String fHost = kName.getHostName();
- final String fServiceName = kName.getServiceName();
- LOG.debug("Server with host: {}", fHost);
+ KerberosName kerberosName = new KerberosName(p.getName());
+ final String hostName = kerberosName.getHostName();
+ final String serviceName = kerberosName.getServiceName();
+ LOG.debug("Server with host: {}", hostName);
saslServer =
Subject.doAs(subject, new PrivilegedExceptionAction<SaslServer>() {
@Override
@@ -104,8 +104,8 @@
props.put(Sasl.QOP, "auth");
props.put(Sasl.SERVER_AUTH, "false");
return Sasl.createSaslServer(SaslUtils.KERBEROS,
- fServiceName,
- fHost, props, fch);
+ serviceName,
+ hostName, props, fch);
} catch (Exception e) {
LOG.error("Subject failed to create sasl server.", e);
return null;
@@ -159,12 +159,12 @@
}
/**
- * CallbackHandler for SASL DIGEST-MD5 mechanism
+ * CallbackHandler for SASL DIGEST-MD5 mechanism.
*/
public static class KerberosSaslCallbackHandler implements CallbackHandler {
/**
- * Used to authenticate the clients
+ * Used to authenticate the clients.
*/
private List<String> authorizedUsers;
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
index 1816ffc..77cb864 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslNettyServerState.java
@@ -16,5 +16,6 @@
final class KerberosSaslNettyServerState {
- public static final AttributeKey<KerberosSaslNettyServer> KERBOROS_SASL_NETTY_SERVER = AttributeKey.valueOf("kerboros.sasl.netty.server");
+ public static final AttributeKey<KerberosSaslNettyServer> KERBOROS_SASL_NETTY_SERVER =
+ AttributeKey.valueOf("kerboros.sasl.netty.server");
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
index 0356538..c227448 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/KerberosSaslServerHandler.java
@@ -30,14 +30,14 @@
* Used for client or server's token to send or receive from each other.
*/
private final Map<String, Object> topoConf;
- private final String jaas_section;
+ private final String jaasSection;
private final List<String> authorizedUsers;
- public KerberosSaslServerHandler(ISaslServer server, Map<String, Object> topoConf, String jaas_section,
+ public KerberosSaslServerHandler(ISaslServer server, Map<String, Object> topoConf, String jaasSection,
List<String> authorizedUsers) throws IOException {
this.server = server;
this.topoConf = topoConf;
- this.jaas_section = jaas_section;
+ this.jaasSection = jaasSection;
this.authorizedUsers = authorizedUsers;
}
@@ -60,7 +60,7 @@
if (saslNettyServer == null) {
LOG.debug("No saslNettyServer for {} yet; creating now, with topology token: ", channel);
try {
- saslNettyServer = new KerberosSaslNettyServer(topoConf, jaas_section, authorizedUsers);
+ saslNettyServer = new KerberosSaslNettyServer(topoConf, jaasSection, authorizedUsers);
channel.attr(KerberosSaslNettyServerState.KERBOROS_SASL_NETTY_SERVER).set(saslNettyServer);
} catch (RuntimeException ioe) {
LOG.error("Error occurred while creating saslNettyServer on server {} for client {}",
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
index f9e42f0..6a50d84 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Login.java
@@ -40,19 +40,21 @@
// and try to renew the ticket.
private static final float TICKET_RENEW_WINDOW = 0.80f;
/**
- * Percentage of random jitter added to the renewal time
+ * Percentage of random jitter added to the renewal time.
*/
private static final float TICKET_RENEW_JITTER = 0.05f;
// Regardless of TICKET_RENEW_WINDOW setting above and the ticket expiry time,
// thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute).
// Change the '1' to e.g. 5, to change this to 5 minutes.
private static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L;
- /** Random number generator */
+ /**
+ * Random number generator.
+ */
private static Random rng = new Random();
public CallbackHandler callbackHandler;
- Logger LOG = Logger.getLogger(Login.class);
+ private static final Logger LOG = Logger.getLogger(Login.class);
private Subject subject = null;
- private Thread t = null;
+ private Thread thread = null;
private boolean isKrbTicket = false;
private boolean isUsingTicketCache = false;
private boolean isUsingKeytab = false;
@@ -82,7 +84,7 @@
this.loginContextName = loginContextName;
subject = login.getSubject();
isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty();
- AppConfigurationEntry entries[] = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+ AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
for (AppConfigurationEntry entry : entries) {
// there will only be a single entry, so this for() loop will only be iterated through once.
if (entry.getOptions().get("useTicketCache") != null) {
@@ -110,7 +112,7 @@
// TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. For testing and development,
// you can decrease the interval of expiration of tickets (for example, to 3 minutes) by running :
// "modprinc -maxlife 3mins <principal>" in kadmin.
- t = new Thread(new Runnable() {
+ thread = new Thread(new Runnable() {
@Override
public void run() {
LOG.info("TGT refresh thread started.");
@@ -128,21 +130,21 @@
long expiry = tgt.getEndTime().getTime();
Date expiryDate = new Date(expiry);
if ((isUsingTicketCache) && (tgt.getEndTime().equals(tgt.getRenewTill()))) {
- LOG.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." +
- "This process will not be able to authenticate new SASL connections after that " +
- "time (for example, it will not be authenticate a new connection with a Zookeeper " +
- "Quorum member). Ask your system administrator to either increase the " +
- "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within " +
- "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " +
- "expiry cannot be further extended by refreshing, exiting refresh thread now.");
+ LOG.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "."
+ + "This process will not be able to authenticate new SASL connections after that "
+ + "time (for example, it will not be authenticate a new connection with a Zookeeper "
+ + "Quorum member). Ask your system administrator to either increase the "
+ + "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within "
+ + "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's "
+ + "expiry cannot be further extended by refreshing, exiting refresh thread now.");
return;
}
// determine how long to sleep from looking at ticket's expiry.
// We should not allow the ticket to expire, but we should take into consideration
// MIN_TIME_BEFORE_RELOGIN. Will not sleep less than MIN_TIME_BEFORE_RELOGIN, unless doing so
// would cause ticket expiration.
- if ((nextRefresh > expiry) ||
- ((now + MIN_TIME_BEFORE_RELOGIN) > expiry)) {
+ if ((nextRefresh > expiry)
+ || ((now + MIN_TIME_BEFORE_RELOGIN) > expiry)) {
// expiry is before next scheduled refresh).
nextRefresh = now;
} else {
@@ -160,8 +162,9 @@
if (tgt != null && now > tgt.getEndTime().getTime()) {
if ((now - tgt.getEndTime().getTime()) < (10 * MIN_TIME_BEFORE_RELOGIN)) {
Date until = new Date(now + MIN_TIME_BEFORE_RELOGIN);
- LOG.info("TGT already expired but giving additional 10 minutes past TGT expiry, refresh sleeping until: " +
- until.toString());
+ LOG.info("TGT already expired but giving additional 10 minutes past TGT expiry, refresh "
+ + "sleeping until: "
+ + until.toString());
try {
Thread.sleep(MIN_TIME_BEFORE_RELOGIN);
} catch (InterruptedException ie) {
@@ -244,22 +247,22 @@
}
}
});
- t.setName("Refresh-TGT");
- t.setDaemon(true);
+ thread.setName("Refresh-TGT");
+ thread.setDaemon(true);
}
public void startThreadIfNeeded() {
- // thread object 't' will be null if a refresh thread is not needed.
- if (t != null) {
- t.start();
+ // thread object 'thread' will be null if a refresh thread is not needed.
+ if (thread != null) {
+ thread.start();
}
}
public void shutdown() {
- if ((t != null) && (t.isAlive())) {
- t.interrupt();
+ if ((thread != null) && (thread.isAlive())) {
+ thread.interrupt();
try {
- t.join();
+ thread.join();
} catch (InterruptedException e) {
LOG.warn("error while waiting for Login thread to shutdown: " + e);
}
@@ -276,11 +279,11 @@
private synchronized LoginContext login(final String loginContextName) throws LoginException {
if (loginContextName == null) {
- throw new LoginException("loginContext name (JAAS file section header) was null. " +
- "Please check your java.security.login.auth.config (=" +
- System.getProperty("java.security.login.auth.config") +
- ") and your " + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY + "(=" +
- System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") + ")");
+ throw new LoginException("loginContext name (JAAS file section header) was null. "
+ + "Please check your java.security.login.auth.config (="
+ + System.getProperty("java.security.login.auth.config")
+ + ") and your " + ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY + "(="
+ + System.getProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, "Client") + ")");
}
LoginContext loginContext = new LoginContext(loginContextName, callbackHandler);
loginContext.login();
@@ -294,8 +297,8 @@
long expires = tgt.getEndTime().getTime();
LOG.info("TGT valid starting at: " + tgt.getStartTime().toString());
LOG.info("TGT expires: " + tgt.getEndTime().toString());
- long proposedRefresh = start + (long) ((expires - start) *
- (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble())));
+ long proposedRefresh = start + (long) ((expires - start)
+ * (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble())));
if (proposedRefresh > expires) {
// proposedRefresh is too far in the future: it's after ticket expires: simply return now.
return System.currentTimeMillis();
@@ -304,6 +307,7 @@
}
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private synchronized KerberosTicket getTGT() {
Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
for (KerberosTicket ticket : tickets) {
@@ -319,9 +323,8 @@
private void sleepUntilSufficientTimeElapsed() {
long now = System.currentTimeMillis();
if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN) {
- LOG.warn("Not attempting to re-login since the last re-login was " +
- "attempted less than " + (MIN_TIME_BEFORE_RELOGIN / 1000) + " seconds" +
- " before.");
+ LOG.warn("Not attempting to re-login since the last re-login was "
+ + "attempted less than " + (MIN_TIME_BEFORE_RELOGIN / 1000) + " seconds before.");
try {
Thread.sleep(MIN_TIME_BEFORE_RELOGIN - (now - getLastLogin()));
} catch (InterruptedException e) {
@@ -334,7 +337,7 @@
}
/**
- * Returns login object
+ * Returns login object.
* @return login
*/
private LoginContext getLogin() {
@@ -342,8 +345,7 @@
}
/**
- * Set the login object
- * @param login
+ * Set the login object.
*/
private void setLogin(LoginContext login) {
this.login = login;
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
index e29d43c..6b15788 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageBatch.java
@@ -18,14 +18,14 @@
class MessageBatch implements INettySerializable {
- private final int buffer_size;
+ private final int bufferSize;
private final ArrayList<TaskMessage> msgs;
- private int encoded_length;
+ private int encodedLength;
- MessageBatch(int buffer_size) {
- this.buffer_size = buffer_size;
+ MessageBatch(int bufferSize) {
+ this.bufferSize = bufferSize;
msgs = new ArrayList<>();
- encoded_length = ControlMessage.EOB_MESSAGE.encodeLength();
+ encodedLength = ControlMessage.EOB_MESSAGE.encodeLength();
}
void add(TaskMessage msg) {
@@ -34,7 +34,7 @@
}
msgs.add(msg);
- encoded_length += msgEncodeLength(msg);
+ encodedLength += msgEncodeLength(msg);
}
private int msgEncodeLength(TaskMessage taskMsg) {
@@ -50,13 +50,15 @@
}
/**
+ * Check whether full.
* @return true if this batch used up allowed buffer size
*/
boolean isFull() {
- return encoded_length >= buffer_size;
+ return encodedLength >= bufferSize;
}
/**
+ * Check whether empty.
* @return true if this batch doesn't have any messages
*/
boolean isEmpty() {
@@ -64,6 +66,7 @@
}
/**
+ * Get size.
* @return number of msgs in this batch
*/
int size() {
@@ -72,11 +75,11 @@
@Override
public int encodeLength() {
- return encoded_length;
+ return encodedLength;
}
/**
- * create a buffer containing the encoding of this batch
+ * create a buffer containing the encoding of this batch.
*/
@Override
public void write(ByteBuf dest) {
@@ -89,24 +92,24 @@
}
/**
- * write a TaskMessage into a buffer
+ * write a TaskMessage into a buffer.
*
- * Each TaskMessage is encoded as: task ... short(2) len ... int(4) payload ... byte[] *
+ * <p>Each TaskMessage is encoded as: task ... short(2) len ... int(4) payload ... byte[] *
*/
private void writeTaskMessage(ByteBuf buf, TaskMessage message) {
- int payload_len = 0;
+ int payloadLen = 0;
if (message.message() != null) {
- payload_len = message.message().length;
+ payloadLen = message.message().length;
}
- int task_id = message.task();
- if (task_id > Short.MAX_VALUE) {
+ int taskId = message.task();
+ if (taskId > Short.MAX_VALUE) {
throw new RuntimeException("Task ID should not exceed " + Short.MAX_VALUE);
}
- buf.writeShort((short) task_id);
- buf.writeInt(payload_len);
- if (payload_len > 0) {
+ buf.writeShort((short) taskId);
+ buf.writeInt(payloadLen);
+ if (payloadLen > 0) {
buf.writeBytes(message.message());
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
index fcd8f0e..bced87c 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/MessageDecoder.java
@@ -61,13 +61,13 @@
available -= 2;
// case 1: Control message
- ControlMessage ctrl_msg = ControlMessage.mkMessage(code);
- if (ctrl_msg != null) {
+ ControlMessage controlMessage = ControlMessage.mkMessage(code);
+ if (controlMessage != null) {
- if (ctrl_msg == ControlMessage.EOB_MESSAGE) {
+ if (controlMessage == ControlMessage.EOB_MESSAGE) {
continue;
} else {
- out.add(ctrl_msg);
+ out.add(controlMessage);
return;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
index 669c88a..368ffba 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslMessageToken.java
@@ -22,12 +22,7 @@
*/
public class SaslMessageToken implements INettySerializable {
public static final short IDENTIFIER = -500;
-
- /**
- * Class logger
- */
- private static final Logger LOG = LoggerFactory
- .getLogger(SaslMessageToken.class);
+ private static final Logger LOG = LoggerFactory.getLogger(SaslMessageToken.class);
/**
* Used for client or server's token to send or receive from each other.
@@ -50,23 +45,23 @@
}
public static SaslMessageToken read(byte[] serial) {
- ByteBuf sm_buffer = Unpooled.wrappedBuffer(serial);
+ ByteBuf smBuffer = Unpooled.wrappedBuffer(serial);
try {
- short identifier = sm_buffer.readShort();
- int payload_len = sm_buffer.readInt();
- if (identifier != IDENTIFIER) {
- return null;
- }
- byte token[] = new byte[payload_len];
- sm_buffer.readBytes(token, 0, payload_len);
- return new SaslMessageToken(token);
+ short identifier = smBuffer.readShort();
+ int payloadLen = smBuffer.readInt();
+ if (identifier != IDENTIFIER) {
+ return null;
+ }
+ byte[] token = new byte[payloadLen];
+ smBuffer.readBytes(token, 0, payloadLen);
+ return new SaslMessageToken(token);
} finally {
- sm_buffer.release();
+ smBuffer.release();
}
}
/**
- * Read accessor for SASL token
+ * Read accessor for SASL token.
*
* @return saslToken SASL token
*/
@@ -75,7 +70,7 @@
}
/**
- * Write accessor for SASL token
+ * Write accessor for SASL token.
*
* @param token SASL token
*/
@@ -96,15 +91,15 @@
*/
@Override
public void write(ByteBuf dest) {
- int payload_len = 0;
+ int payloadLen = 0;
if (token != null) {
- payload_len = token.length;
+ payloadLen = token.length;
}
dest.writeShort(IDENTIFIER);
- dest.writeInt(payload_len);
+ dest.writeInt(payloadLen);
- if (payload_len > 0) {
+ if (payloadLen > 0) {
dest.writeBytes(token);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
index 11f487d..fe4de08 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyClient.java
@@ -85,11 +85,11 @@
*/
private static class SaslClientCallbackHandler implements CallbackHandler {
/**
- * Generated username contained in TopologyToken
+ * Generated username contained in TopologyToken.
*/
private final String userName;
/**
- * Generated password contained in TopologyToken
+ * Generated password contained in TopologyToken.
*/
private final char[] userPassword;
@@ -106,7 +106,6 @@
* Implementation used to respond to SASL tokens from server.
*
* @param callbacks objects that indicate what credential information the server's SaslServer requires from the client.
- * @throws UnsupportedCallbackException
*/
@Override
public void handle(Callback[] callbacks)
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
index 06763e2..6c04d76 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslNettyServer.java
@@ -78,12 +78,12 @@
}
/**
- * CallbackHandler for SASL DIGEST-MD5 mechanism
+ * CallbackHandler for SASL DIGEST-MD5 mechanism.
*/
public static class SaslDigestCallbackHandler implements CallbackHandler {
/**
- * Used to authenticate the clients
+ * Used to authenticate the clients.
*/
private byte[] userPassword;
private String userName;
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormClientHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormClientHandler.java
index 25b0aa2..f1191a2 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormClientHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormClientHandler.java
@@ -24,7 +24,7 @@
private static final Logger LOG = LoggerFactory
.getLogger(SaslStormClientHandler.class);
- private final long start_time;
+ private final long startTime;
private final ISaslClient client;
/**
* Used for client or server's token to send or receive from each other.
@@ -34,7 +34,7 @@
public SaslStormClientHandler(ISaslClient client) throws IOException {
this.client = client;
- start_time = System.currentTimeMillis();
+ startTime = System.currentTimeMillis();
getSASLCredentials();
}
@@ -64,7 +64,7 @@
@Override
public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception {
LOG.debug("send/recv time (ms): {}",
- (System.currentTimeMillis() - start_time));
+ (System.currentTimeMillis() - startTime));
// examine the response message from server
if (message instanceof ControlMessage) {
@@ -145,6 +145,7 @@
channel.writeAndFlush(saslResponse, channel.voidPromise());
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private void getSASLCredentials() throws IOException {
String secretKey;
name = client.name();
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormServerHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormServerHandler.java
index ce69a6f..181667b 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormServerHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/SaslStormServerHandler.java
@@ -128,6 +128,7 @@
ctx.close();
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private void getSASLCredentials() throws IOException {
String secretKey;
topologyName = server.name();
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java
index a3cd8b0..8a37d1e 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/Server.java
@@ -67,7 +67,7 @@
private volatile boolean closing = false;
/**
- * Starts Netty at the given port
+ * Starts Netty at the given port.
* @param topoConf The topology config
* @param port The port to start Netty at
* @param cb The callback to deliver incoming messages to
@@ -81,8 +81,7 @@
this.newConnectionResponse = newConnectionResponse;
// Configure the server.
- int buffer_size = ObjectReader.getInt(topoConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
- int backlog = ObjectReader.getInt(topoConf.get(Config.STORM_MESSAGING_NETTY_SOCKET_BACKLOG), 500);
+ int bufferSize = ObjectReader.getInt(topoConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
int maxWorkers = ObjectReader.getInt(topoConf.get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS));
ThreadFactory bossFactory = new NettyRenameThreadFactory(netty_name() + "-boss");
@@ -93,15 +92,16 @@
// https://github.com/netty/netty/blob/netty-4.1.24.Final/transport/src/main/java/io/netty/channel/MultithreadEventLoopGroup.java#L40
this.workerEventLoopGroup = new NioEventLoopGroup(maxWorkers > 0 ? maxWorkers : 0, workerFactory);
- LOG.info("Create Netty Server " + netty_name() + ", buffer_size: " + buffer_size + ", maxWorkers: " + maxWorkers);
+ LOG.info("Create Netty Server " + netty_name() + ", buffer_size: " + bufferSize + ", maxWorkers: " + maxWorkers);
+ int backlog = ObjectReader.getInt(topoConf.get(Config.STORM_MESSAGING_NETTY_SOCKET_BACKLOG), 500);
bootstrap = new ServerBootstrap()
.group(bossEventLoopGroup, workerEventLoopGroup)
.channel(NioServerSocketChannel.class)
.option(ChannelOption.SO_REUSEADDR, true)
.option(ChannelOption.SO_BACKLOG, backlog)
.childOption(ChannelOption.TCP_NODELAY, true)
- .childOption(ChannelOption.SO_RCVBUF, buffer_size)
+ .childOption(ChannelOption.SO_RCVBUF, bufferSize)
.childOption(ChannelOption.SO_KEEPALIVE, true)
.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
.childHandler(new StormServerPipelineFactory(topoConf, this));
@@ -136,9 +136,7 @@
}
/**
- * enqueue a received message
- *
- * @throws InterruptedException
+ * enqueue a received message.
*/
protected void enqueue(List<TaskMessage> msgs, String from) throws InterruptedException {
if (null == msgs || msgs.isEmpty() || closing) {
@@ -154,7 +152,7 @@
}
/**
- * close all channels, and release resources
+ * close all channels, and release resources.
*/
@Override
public void close() {
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientHandler.java
index 90d03d6..a3837a8 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/StormClientHandler.java
@@ -27,13 +27,13 @@
public class StormClientHandler extends ChannelInboundHandlerAdapter {
private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class);
private final Client client;
- private final KryoValuesDeserializer _des;
+ private final KryoValuesDeserializer des;
private final AtomicBoolean[] remoteBpStatus;
StormClientHandler(Client client, AtomicBoolean[] remoteBpStatus, Map<String, Object> conf) {
this.client = client;
this.remoteBpStatus = remoteBpStatus;
- _des = new KryoValuesDeserializer(conf);
+ des = new KryoValuesDeserializer(conf);
}
@Override
@@ -80,7 +80,7 @@
if (tm.task() != Server.LOAD_METRICS_TASK_ID) {
throw new RuntimeException("Metrics messages are sent to the system task (" + client.getDstAddress() + ") " + tm);
}
- List<Object> metrics = _des.deserialize(tm.message());
+ List<Object> metrics = des.deserialize(tm.message());
if (metrics.size() < 1) {
throw new RuntimeException("No metrics data in the metrics message (" + client.getDstAddress() + ") " + metrics);
}
diff --git a/storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerHandler.java b/storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerHandler.java
index 3c256bb..542dd9c 100644
--- a/storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/messaging/netty/StormServerHandler.java
@@ -28,11 +28,11 @@
private static final Logger LOG = LoggerFactory.getLogger(StormServerHandler.class);
private static final Set<Class<?>> ALLOWED_EXCEPTIONS = new HashSet<>(Arrays.asList(new Class<?>[]{ IOException.class }));
private final IServer server;
- private final AtomicInteger failure_count;
+ private final AtomicInteger failureCount;
public StormServerHandler(IServer server) {
this.server = server;
- failure_count = new AtomicInteger(0);
+ failureCount = new AtomicInteger(0);
}
@Override
@@ -51,7 +51,7 @@
server.received(msg, channel.remoteAddress().toString(), channel);
} catch (InterruptedException e) {
LOG.info("failed to enqueue a request message", e);
- failure_count.incrementAndGet();
+ failureCount.incrementAndGet();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/EventLoggerBolt.java b/storm-client/src/jvm/org/apache/storm/metric/EventLoggerBolt.java
index a66fe47..9776848 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/EventLoggerBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/EventLoggerBolt.java
@@ -12,6 +12,10 @@
package org.apache.storm.metric;
+import static org.apache.storm.daemon.StormCommon.TOPOLOGY_EVENT_LOGGER_ARGUMENTS;
+import static org.apache.storm.daemon.StormCommon.TOPOLOGY_EVENT_LOGGER_CLASS;
+import static org.apache.storm.metric.IEventLogger.EventInfo;
+
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -23,10 +27,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.daemon.StormCommon.TOPOLOGY_EVENT_LOGGER_ARGUMENTS;
-import static org.apache.storm.daemon.StormCommon.TOPOLOGY_EVENT_LOGGER_CLASS;
-import static org.apache.storm.metric.IEventLogger.EventInfo;
-
public class EventLoggerBolt implements IBolt {
/*
diff --git a/storm-client/src/jvm/org/apache/storm/metric/IEventLogger.java b/storm-client/src/jvm/org/apache/storm/metric/IEventLogger.java
index 76b9b41..7a8af67 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/IEventLogger.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/IEventLogger.java
@@ -73,7 +73,7 @@
}
/**
- * Returns a default formatted string with fields separated by ","
+ * Returns a default formatted string with fields separated by ",".
*
* @return a default formatted string with fields separated by ","
*/
diff --git a/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java b/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java
index 2e722b2..769163e 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java
@@ -23,17 +23,17 @@
/**
* Listens for all metrics, dumps them to log
*
- * To use, add this to your topology's configuration:
+ * <p>To use, add this to your topology's configuration:
*
- * ```java conf.registerMetricsConsumer(org.apache.storm.metrics.LoggingMetricsConsumer.class, 1); ```
+ * <p>```java conf.registerMetricsConsumer(org.apache.storm.metrics.LoggingMetricsConsumer.class, 1); ```
*
- * Or edit the storm.yaml config file:
+ * <p>Or edit the storm.yaml config file:
*
- * ```yaml topology.metrics.consumer.register: - class: "org.apache.storm.metrics.LoggingMetricsConsumer" parallelism.hint: 1 ```
+ * <p>```yaml topology.metrics.consumer.register: - class: "org.apache.storm.metrics.LoggingMetricsConsumer" parallelism.hint: 1 ```
*/
public class LoggingMetricsConsumer implements IMetricsConsumer {
public static final Logger LOG = LoggerFactory.getLogger(LoggingMetricsConsumer.class);
- static private String padding = " ";
+ private static String padding = " ";
@Override
public void prepare(Map<String, Object> topoConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) {
@@ -51,8 +51,10 @@
for (DataPoint p : dataPoints) {
sb.delete(header.length(), sb.length());
sb.append(p.name)
- .append(padding).delete(header.length() + 23, sb.length()).append("\t")
- .append(p.value);
+ .append(padding)
+ .delete(header.length() + 23, sb.length())
+ .append("\t")
+ .append(p.value);
LOG.info(sb.toString());
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/MetricsConsumerBolt.java b/storm-client/src/jvm/org/apache/storm/metric/MetricsConsumerBolt.java
index 46bac22..e76868a 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/MetricsConsumerBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/MetricsConsumerBolt.java
@@ -32,72 +32,75 @@
public class MetricsConsumerBolt implements IBolt {
public static final Logger LOG = LoggerFactory.getLogger(MetricsConsumerBolt.class);
- private final int _maxRetainMetricTuples;
- private final Predicate<IMetricsConsumer.DataPoint> _filterPredicate;
- private final DataPointExpander _expander;
- private final BlockingQueue<MetricsTask> _taskQueue;
- IMetricsConsumer _metricsConsumer;
- String _consumerClassName;
- OutputCollector _collector;
- Object _registrationArgument;
- private Thread _taskExecuteThread;
- private volatile boolean _running = true;
+ private final int maxRetainMetricTuples;
+ private final Predicate<IMetricsConsumer.DataPoint> filterPredicate;
+ private final DataPointExpander expander;
+ private final BlockingQueue<MetricsTask> taskQueue;
+ IMetricsConsumer metricsConsumer;
+ String consumerClassName;
+ OutputCollector collector;
+ Object registrationArgument;
+ private Thread taskExecuteThread;
+ private volatile boolean running = true;
public MetricsConsumerBolt(String consumerClassName, Object registrationArgument, int maxRetainMetricTuples,
Predicate<IMetricsConsumer.DataPoint> filterPredicate, DataPointExpander expander) {
- _consumerClassName = consumerClassName;
- _registrationArgument = registrationArgument;
- _maxRetainMetricTuples = maxRetainMetricTuples;
- _filterPredicate = filterPredicate;
- _expander = expander;
+ this.consumerClassName = consumerClassName;
+ this.registrationArgument = registrationArgument;
+ this.maxRetainMetricTuples = maxRetainMetricTuples;
+ this.filterPredicate = filterPredicate;
+ this.expander = expander;
- if (_maxRetainMetricTuples > 0) {
- _taskQueue = new LinkedBlockingDeque<>(_maxRetainMetricTuples);
+ if (this.maxRetainMetricTuples > 0) {
+ taskQueue = new LinkedBlockingDeque<>(this.maxRetainMetricTuples);
} else {
- _taskQueue = new LinkedBlockingDeque<>();
+ taskQueue = new LinkedBlockingDeque<>();
}
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
try {
- _metricsConsumer = (IMetricsConsumer) Class.forName(_consumerClassName).newInstance();
+ metricsConsumer = (IMetricsConsumer) Class.forName(consumerClassName).newInstance();
} catch (Exception e) {
- throw new RuntimeException("Could not instantiate a class listed in config under section " +
- Config.TOPOLOGY_METRICS_CONSUMER_REGISTER + " with fully qualified name " + _consumerClassName, e);
+ throw new RuntimeException("Could not instantiate a class listed in config under section "
+ + Config.TOPOLOGY_METRICS_CONSUMER_REGISTER
+ + " with fully qualified name "
+ + consumerClassName,
+ e);
}
- _metricsConsumer.prepare(topoConf, _registrationArgument, context, collector);
- _collector = collector;
- _taskExecuteThread = new Thread(new MetricsHandlerRunnable());
- _taskExecuteThread.setDaemon(true);
- _taskExecuteThread.start();
+ metricsConsumer.prepare(topoConf, registrationArgument, context, collector);
+ this.collector = collector;
+ taskExecuteThread = new Thread(new MetricsHandlerRunnable());
+ taskExecuteThread.setDaemon(true);
+ taskExecuteThread.start();
}
@Override
public void execute(Tuple input) {
IMetricsConsumer.TaskInfo taskInfo = (IMetricsConsumer.TaskInfo) input.getValue(0);
Collection<IMetricsConsumer.DataPoint> dataPoints = (Collection) input.getValue(1);
- Collection<IMetricsConsumer.DataPoint> expandedDataPoints = _expander.expandDataPoints(dataPoints);
+ Collection<IMetricsConsumer.DataPoint> expandedDataPoints = expander.expandDataPoints(dataPoints);
List<IMetricsConsumer.DataPoint> filteredDataPoints = getFilteredDataPoints(expandedDataPoints);
MetricsTask metricsTask = new MetricsTask(taskInfo, filteredDataPoints);
- while (!_taskQueue.offer(metricsTask)) {
- _taskQueue.poll();
+ while (!taskQueue.offer(metricsTask)) {
+ taskQueue.poll();
}
- _collector.ack(input);
+ collector.ack(input);
}
private List<IMetricsConsumer.DataPoint> getFilteredDataPoints(Collection<IMetricsConsumer.DataPoint> dataPoints) {
- return Lists.newArrayList(Iterables.filter(dataPoints, _filterPredicate));
+ return Lists.newArrayList(Iterables.filter(dataPoints, filterPredicate));
}
@Override
public void cleanup() {
- _running = false;
- _metricsConsumer.cleanup();
- _taskExecuteThread.interrupt();
+ running = false;
+ metricsConsumer.cleanup();
+ taskExecuteThread.interrupt();
}
static class MetricsTask {
@@ -122,10 +125,10 @@
@Override
public void run() {
- while (_running) {
+ while (running) {
try {
- MetricsTask task = _taskQueue.take();
- _metricsConsumer.handleDataPoints(task.getTaskInfo(), task.getDataPoints());
+ MetricsTask task = taskQueue.take();
+ metricsConsumer.handleDataPoints(task.getTaskInfo(), task.getDataPoints());
} catch (InterruptedException e) {
break;
} catch (Throwable t) {
diff --git a/storm-client/src/jvm/org/apache/storm/metric/SystemBolt.java b/storm-client/src/jvm/org/apache/storm/metric/SystemBolt.java
index a168230..5e978a7 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/SystemBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/SystemBolt.java
@@ -30,26 +30,25 @@
import org.apache.storm.utils.ObjectReader;
import org.apache.storm.utils.ReflectionUtils;
-
// There is one task inside one executor for each worker of the topology.
// TaskID is always -1, therefore you can only send-unanchored tuples to co-located SystemBolt.
// This bolt was conceived to export worker stats via metrics api.
public class SystemBolt implements IBolt {
- private static boolean _prepareWasCalled = false;
+ private static boolean prepareWasCalled = false;
@SuppressWarnings({ "unchecked" })
@Override
public void prepare(final Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- if (_prepareWasCalled && !"local".equals(topoConf.get(Config.STORM_CLUSTER_MODE))) {
+ if (prepareWasCalled && !"local".equals(topoConf.get(Config.STORM_CLUSTER_MODE))) {
throw new RuntimeException("A single worker should have 1 SystemBolt instance.");
}
- _prepareWasCalled = true;
+ prepareWasCalled = true;
int bucketSize = ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS));
- final RuntimeMXBean jvmRT = ManagementFactory.getRuntimeMXBean();
- context.registerMetric("uptimeSecs", () -> jvmRT.getUptime() / 1000.0, bucketSize);
- context.registerMetric("startTimeSecs", () -> jvmRT.getStartTime() / 1000.0, bucketSize);
+ final RuntimeMXBean jvmRt = ManagementFactory.getRuntimeMXBean();
+ context.registerMetric("uptimeSecs", () -> jvmRt.getUptime() / 1000.0, bucketSize);
+ context.registerMetric("startTimeSecs", () -> jvmRt.getStartTime() / 1000.0, bucketSize);
final ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
context.registerMetric("threadCount", threadBean::getThreadCount, bucketSize);
@@ -68,6 +67,7 @@
}
}, bucketSize);
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
final MemoryMXBean jvmMemRT = ManagementFactory.getMemoryMXBean();
context.registerMetric("memory/heap", new MemoryUsageMetric(jvmMemRT::getHeapMemoryUsage), bucketSize);
@@ -104,15 +104,15 @@
}
private static class MemoryUsageMetric implements IMetric {
- Supplier<MemoryUsage> _getUsage;
+ Supplier<MemoryUsage> getUsage;
public MemoryUsageMetric(Supplier<MemoryUsage> getUsage) {
- _getUsage = getUsage;
+ this.getUsage = getUsage;
}
@Override
public Object getValueAndReset() {
- MemoryUsage memUsage = _getUsage.get();
+ MemoryUsage memUsage = getUsage.get();
HashMap<String, Object> m = new HashMap<>();
m.put("maxBytes", memUsage.getMax());
m.put("committedBytes", memUsage.getCommitted());
@@ -127,28 +127,28 @@
// canonically the metrics data exported is time bucketed when doing counts.
// convert the absolute values here into time buckets.
private static class GarbageCollectorMetric implements IMetric {
- GarbageCollectorMXBean _gcBean;
- Long _collectionCount;
- Long _collectionTime;
+ GarbageCollectorMXBean gcBean;
+ Long collectionCount;
+ Long collectionTime;
public GarbageCollectorMetric(GarbageCollectorMXBean gcBean) {
- _gcBean = gcBean;
+ this.gcBean = gcBean;
}
@Override
public Object getValueAndReset() {
- Long collectionCountP = _gcBean.getCollectionCount();
- Long collectionTimeP = _gcBean.getCollectionTime();
+ Long collectionCountP = gcBean.getCollectionCount();
+ Long collectionTimeP = gcBean.getCollectionTime();
Map<String, Object> ret = null;
- if (_collectionCount != null && _collectionTime != null) {
+ if (collectionCount != null && collectionTime != null) {
ret = new HashMap<>();
- ret.put("count", collectionCountP - _collectionCount);
- ret.put("timeMs", collectionTimeP - _collectionTime);
+ ret.put("count", collectionCountP - collectionCount);
+ ret.put("timeMs", collectionTimeP - collectionTime);
}
- _collectionCount = collectionCountP;
- _collectionTime = collectionTimeP;
+ collectionCount = collectionCountP;
+ collectionTime = collectionTimeP;
return ret;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java
index 8e980ad..8bcc40b 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java
@@ -13,18 +13,18 @@
package org.apache.storm.metric.api;
public class AssignableMetric implements IMetric {
- Object _value;
+ Object value;
public AssignableMetric(Object value) {
- _value = value;
+ this.value = value;
}
public void setValue(Object value) {
- _value = value;
+ this.value = value;
}
@Override
public Object getValueAndReset() {
- return _value;
+ return value;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java
index 944533f..9b84a98 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java
@@ -13,22 +13,22 @@
package org.apache.storm.metric.api;
public class CombinedMetric implements IMetric {
- private final ICombiner _combiner;
- private Object _value;
+ private final ICombiner combiner;
+ private Object value;
public CombinedMetric(ICombiner combiner) {
- _combiner = combiner;
- _value = _combiner.identity();
+ this.combiner = combiner;
+ value = this.combiner.identity();
}
public void update(Object value) {
- _value = _combiner.combine(_value, value);
+ this.value = combiner.combine(this.value, value);
}
@Override
public Object getValueAndReset() {
- Object ret = _value;
- _value = _combiner.identity();
+ Object ret = value;
+ value = combiner.identity();
return ret;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java
index 8284ed9..36df8f9 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java
@@ -13,23 +13,23 @@
package org.apache.storm.metric.api;
public class CountMetric implements IMetric {
- long _value = 0;
+ long value = 0;
public CountMetric() {
}
public void incr() {
- _value++;
+ value++;
}
public void incrBy(long incrementBy) {
- _value += incrementBy;
+ value += incrementBy;
}
@Override
public Object getValueAndReset() {
- long ret = _value;
- _value = 0;
+ long ret = value;
+ value = 0;
return ret;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
index 4c45f0d..c6f86f4 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
@@ -17,6 +17,7 @@
*/
public interface IMetric {
/**
+ * Get value and reset.
* @return an object that will be sent sent to {@link IMetricsConsumer#handleDataPoints(org.apache.storm.metric.api.IMetricsConsumer
* .TaskInfo,
* java.util.Collection)}. If null is returned nothing will be sent. If this value can be reset, like with a counter, a side effect
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java b/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java
index 1558354..a332dc4 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java
@@ -48,8 +48,10 @@
@Override
public String toString() {
- return "TASK_INFO: { host: " + srcWorkerHost + ":" + srcWorkerPort +
- " comp: " + srcComponentId + "[" + srcTaskId + "]}";
+ return "TASK_INFO: { host: " + srcWorkerHost
+ + ":" + srcWorkerPort
+ + " comp: " + srcComponentId
+ + "[" + srcTaskId + "]}";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java b/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java
index e78ddb8..d60015b 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java
@@ -18,11 +18,6 @@
package org.apache.storm.metric.api;
-class MeanReducerState {
- public int count = 0;
- public double sum = 0.0;
-}
-
public class MeanReducer implements IReducer<MeanReducerState> {
@Override
public MeanReducerState init() {
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducerState.java b/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducerState.java
new file mode 100644
index 0000000..451bbb0
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducerState.java
@@ -0,0 +1,24 @@
+/*
+ * 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.metric.api;
+
+class MeanReducerState {
+ public int count = 0;
+ public double sum = 0.0;
+}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java
index cc2dce2..d133665 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java
@@ -16,15 +16,15 @@
import java.util.Map;
public class MultiCountMetric implements IMetric {
- Map<String, CountMetric> _value = new HashMap<>();
+ Map<String, CountMetric> value = new HashMap<>();
public MultiCountMetric() {
}
public CountMetric scope(String key) {
- CountMetric val = _value.get(key);
+ CountMetric val = value.get(key);
if (val == null) {
- _value.put(key, val = new CountMetric());
+ value.put(key, val = new CountMetric());
}
return val;
}
@@ -32,7 +32,7 @@
@Override
public Map<String, Object> getValueAndReset() {
Map<String, Object> ret = new HashMap<>();
- for (Map.Entry<String, CountMetric> e : _value.entrySet()) {
+ for (Map.Entry<String, CountMetric> e : value.entrySet()) {
ret.put(e.getKey(), e.getValue().getValueAndReset());
}
return ret;
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java
index 079b320..c9c8590 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java
@@ -16,17 +16,17 @@
import java.util.Map;
public class MultiReducedMetric implements IMetric {
- Map<String, ReducedMetric> _value = new HashMap<>();
- IReducer _reducer;
+ Map<String, ReducedMetric> value = new HashMap<>();
+ IReducer reducer;
public MultiReducedMetric(IReducer reducer) {
- _reducer = reducer;
+ this.reducer = reducer;
}
public ReducedMetric scope(String key) {
- ReducedMetric val = _value.get(key);
+ ReducedMetric val = value.get(key);
if (val == null) {
- _value.put(key, val = new ReducedMetric(_reducer));
+ value.put(key, val = new ReducedMetric(reducer));
}
return val;
}
@@ -34,7 +34,7 @@
@Override
public Map<String, Object> getValueAndReset() {
Map<String, Object> ret = new HashMap<>();
- for (Map.Entry<String, ReducedMetric> e : _value.entrySet()) {
+ for (Map.Entry<String, ReducedMetric> e : value.entrySet()) {
Object val = e.getValue().getValueAndReset();
if (val != null) {
ret.put(e.getKey(), val);
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java
index 92ee6ff..718d34f 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java
@@ -13,22 +13,22 @@
package org.apache.storm.metric.api;
public class ReducedMetric implements IMetric {
- private final IReducer _reducer;
- private Object _accumulator;
+ private final IReducer reducer;
+ private Object accumulator;
public ReducedMetric(IReducer reducer) {
- _reducer = reducer;
- _accumulator = _reducer.init();
+ this.reducer = reducer;
+ accumulator = this.reducer.init();
}
public void update(Object value) {
- _accumulator = _reducer.reduce(_accumulator, value);
+ accumulator = reducer.reduce(accumulator, value);
}
@Override
public Object getValueAndReset() {
- Object ret = _reducer.extractResult(_accumulator);
- _accumulator = _reducer.init();
+ Object ret = reducer.extractResult(accumulator);
+ accumulator = reducer.init();
return ret;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/StateMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/StateMetric.java
index 1234bda..cde20b4 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/StateMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/StateMetric.java
@@ -13,14 +13,14 @@
package org.apache.storm.metric.api;
public class StateMetric implements IMetric {
- private IStatefulObject _obj;
+ private IStatefulObject obj;
public StateMetric(IStatefulObject obj) {
- _obj = obj;
+ this.obj = obj;
}
@Override
public Object getValueAndReset() {
- return _obj.getState();
+ return obj.getState();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/rpc/CountShellMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/rpc/CountShellMetric.java
index abc2074..dab9d69 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/rpc/CountShellMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/rpc/CountShellMetric.java
@@ -15,12 +15,15 @@
import org.apache.storm.metric.api.CountMetric;
public class CountShellMetric extends CountMetric implements IShellMetric {
- /***
+ /**
+ * Update metric from RPC.
+ *
* @param value should be null or long
- * if value is null, it will call incr()
- * if value is long, it will call incrBy((long)params)
- * */
+ * if value is null, it will call incr()
+ * if value is long, it will call incrBy((long)params)
+ */
@Override
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void updateMetricFromRPC(Object value) {
if (value == null) {
incr();
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/rpc/IShellMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/rpc/IShellMetric.java
index 8de34ba..a66e178 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/rpc/IShellMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/rpc/IShellMetric.java
@@ -15,12 +15,13 @@
import org.apache.storm.metric.api.IMetric;
public interface IShellMetric extends IMetric {
- /***
- * @function
- * This interface is used by ShellBolt and ShellSpout through RPC call to update Metric
+ /**
+ * This interface is used by ShellBolt and ShellSpout through RPC call to update Metric.
+ *
* @param
* value used to update metric, its's meaning change according implementation
* Object can be any json support types: String, Long, Double, Boolean, Null, List, Map
- * */
+ */
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void updateMetricFromRPC(Object value);
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java
index af925eb..eef6546 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java
@@ -23,7 +23,7 @@
import org.apache.storm.container.cgroup.core.CpuacctCore.StatType;
/**
- * Report CPU used in the cgroup
+ * Report CPU used in the cgroup.
*/
public class CGroupCpu extends CGroupMetricsBase<Map<String, Long>> {
long previousSystem = 0;
@@ -34,6 +34,7 @@
super(conf, SubSystemType.cpuacct);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public synchronized int getUserHZ() throws IOException {
if (userHz < 0) {
ProcessBuilder pb = new ProcessBuilder("getconf", "CLK_TCK");
@@ -52,6 +53,7 @@
long systemHz = stat.get(StatType.system);
long userHz = stat.get(StatType.user);
long user = userHz - previousUser;
+ @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
long sys = systemHz - previousSystem;
previousUser = userHz;
previousSystem = systemHz;
diff --git a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java
index adfebe6..d6d5750 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java
@@ -20,12 +20,12 @@
import org.slf4j.LoggerFactory;
/**
- * Reports the current memory limit of the cgroup for this worker
+ * Reports the current memory limit of the cgroup for this worker.
*/
public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
private static final Logger LOG = LoggerFactory.getLogger(CGroupMemoryLimit.class);
private static final long BYTES_PER_MB = 1024 * 1024;
- private final long _workerLimitBytes;
+ private final long workerLimitBytes;
public CGroupMemoryLimit(Map<String, Object> conf) {
super(conf, SubSystemType.memory);
@@ -36,13 +36,13 @@
} catch (NumberFormatException e) {
LOG.warn("Error Parsing worker.memory_limit_mb {}", e);
}
- _workerLimitBytes = BYTES_PER_MB * limit;
+ workerLimitBytes = BYTES_PER_MB * limit;
}
@Override
public Long getDataFrom(CgroupCore core) throws Exception {
- if (_workerLimitBytes > 0) {
- return _workerLimitBytes;
+ if (workerLimitBytes > 0) {
+ return workerLimitBytes;
}
return ((MemoryCore) core).getPhysicalUsageLimit();
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java
index bf33805..e30feae 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java
@@ -18,7 +18,7 @@
import org.apache.storm.container.cgroup.core.MemoryCore;
/**
- * Reports the current memory usage of the cgroup for this worker
+ * Reports the current memory usage of the cgroup for this worker.
*/
public class CGroupMemoryUsage extends CGroupMetricsBase<Long> {
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/CountStatAndMetric.java b/storm-client/src/jvm/org/apache/storm/metric/internal/CountStatAndMetric.java
index d8112db..31c51fd 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/CountStatAndMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/CountStatAndMetric.java
@@ -22,29 +22,31 @@
* Acts as a Count Metric, but also keeps track of approximate counts for the last 10 mins, 3 hours, 1 day, and all time.
*/
public class CountStatAndMetric implements IMetric {
- private final AtomicLong _currentBucket;
+ private final AtomicLong currentBucket;
//10 min values
- private final int _tmSize;
- private final long[] _tmBuckets;
- private final long[] _tmTime;
+ private final int tmSize;
+ private final long[] tmBuckets;
+ private final long[] tmTime;
//3 hour values
- private final int _thSize;
- private final long[] _thBuckets;
- private final long[] _thTime;
+ private final int thSize;
+ private final long[] thBuckets;
+ private final long[] thTime;
//1 day values
- private final int _odSize;
- private final long[] _odBuckets;
- private final long[] _odTime;
- private final TimerTask _task;
+ private final int odSize;
+ private final long[] odBuckets;
+ private final long[] odTime;
+ private final TimerTask task;
// All internal state except for the count of the current bucket are
// protected using a lock on this counter
- private long _bucketStart;
+ private long bucketStart;
//exact variable time, that is added to the current bucket
- private long _exactExtra;
+ private long exactExtra;
//all time
- private long _allTime;
+ private long allTime;
/**
+ * Constructor.
+ *
* @param numBuckets the number of buckets to divide the time periods into.
*/
public CountStatAndMetric(int numBuckets) {
@@ -52,7 +54,7 @@
}
/**
- * Constructor
+ * Constructor.
*
* @param numBuckets the number of buckets to divide the time periods into.
* @param startTime if positive the simulated time to start the from.
@@ -61,28 +63,28 @@
numBuckets = Math.max(numBuckets, 2);
//We want to capture the full time range, so the target size is as
// if we had one bucket less, then we do
- _tmSize = 10 * 60 * 1000 / (numBuckets - 1);
- _thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
- _odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
- if (_tmSize < 1 || _thSize < 1 || _odSize < 1) {
+ tmSize = 10 * 60 * 1000 / (numBuckets - 1);
+ thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
+ odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
+ if (tmSize < 1 || thSize < 1 || odSize < 1) {
throw new IllegalArgumentException("number of buckets is too large to be supported");
}
- _tmBuckets = new long[numBuckets];
- _tmTime = new long[numBuckets];
- _thBuckets = new long[numBuckets];
- _thTime = new long[numBuckets];
- _odBuckets = new long[numBuckets];
- _odTime = new long[numBuckets];
- _allTime = 0;
- _exactExtra = 0;
+ tmBuckets = new long[numBuckets];
+ tmTime = new long[numBuckets];
+ thBuckets = new long[numBuckets];
+ thTime = new long[numBuckets];
+ odBuckets = new long[numBuckets];
+ odTime = new long[numBuckets];
+ allTime = 0;
+ exactExtra = 0;
- _bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
- _currentBucket = new AtomicLong(0);
+ bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
+ currentBucket = new AtomicLong(0);
if (startTime < 0) {
- _task = new Fresher();
- MetricStatTimer._timer.scheduleAtFixedRate(_task, _tmSize, _tmSize);
+ task = new Fresher();
+ MetricStatTimer.timer.scheduleAtFixedRate(task, tmSize, tmSize);
} else {
- _task = null;
+ task = null;
}
}
@@ -92,7 +94,7 @@
* @param count number to count
*/
public void incBy(long count) {
- _currentBucket.addAndGet(count);
+ currentBucket.addAndGet(count);
}
@@ -101,29 +103,30 @@
return getValueAndReset(System.currentTimeMillis());
}
+ @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
synchronized Object getValueAndReset(long now) {
- long value = _currentBucket.getAndSet(0);
- long timeSpent = now - _bucketStart;
- long ret = value + _exactExtra;
- _bucketStart = now;
- _exactExtra = 0;
+ long value = currentBucket.getAndSet(0);
+ long timeSpent = now - bucketStart;
+ long ret = value + exactExtra;
+ bucketStart = now;
+ exactExtra = 0;
rotateBuckets(value, timeSpent);
return ret;
}
synchronized void rotateSched(long now) {
- long value = _currentBucket.getAndSet(0);
- long timeSpent = now - _bucketStart;
- _exactExtra += value;
- _bucketStart = now;
+ long value = currentBucket.getAndSet(0);
+ long timeSpent = now - bucketStart;
+ exactExtra += value;
+ bucketStart = now;
rotateBuckets(value, timeSpent);
}
synchronized void rotateBuckets(long value, long timeSpent) {
- rotate(value, timeSpent, _tmSize, _tmTime, _tmBuckets);
- rotate(value, timeSpent, _thSize, _thTime, _thBuckets);
- rotate(value, timeSpent, _odSize, _odTime, _odBuckets);
- _allTime += value;
+ rotate(value, timeSpent, tmSize, tmTime, tmBuckets);
+ rotate(value, timeSpent, thSize, thTime, thBuckets);
+ rotate(value, timeSpent, odSize, odTime, odBuckets);
+ allTime += value;
}
private synchronized void rotate(long value, long timeSpent, long targetSize, long[] times, long[] buckets) {
@@ -146,6 +149,7 @@
}
/**
+ * Get time counts.
* @return a map of time window to count. Keys are "600" for last 10 mins "10800" for the last 3 hours "86400" for the last day
* ":all-time" for all time
*/
@@ -155,12 +159,12 @@
synchronized Map<String, Long> getTimeCounts(long now) {
Map<String, Long> ret = new HashMap<>();
- long value = _currentBucket.get();
- long timeSpent = now - _bucketStart;
- ret.put("600", readApproximateTime(value, timeSpent, _tmTime, _tmBuckets, 600 * 1000));
- ret.put("10800", readApproximateTime(value, timeSpent, _thTime, _thBuckets, 10800 * 1000));
- ret.put("86400", readApproximateTime(value, timeSpent, _odTime, _odBuckets, 86400 * 1000));
- ret.put(":all-time", value + _allTime);
+ long value = currentBucket.get();
+ long timeSpent = now - bucketStart;
+ ret.put("600", readApproximateTime(value, timeSpent, tmTime, tmBuckets, 600 * 1000));
+ ret.put("10800", readApproximateTime(value, timeSpent, thTime, thBuckets, 10800 * 1000));
+ ret.put("86400", readApproximateTime(value, timeSpent, odTime, odBuckets, 86400 * 1000));
+ ret.put(":all-time", value + allTime);
return ret;
}
@@ -181,8 +185,8 @@
}
public void close() {
- if (_task != null) {
- _task.cancel();
+ if (task != null) {
+ task.cancel();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/LatencyStatAndMetric.java b/storm-client/src/jvm/org/apache/storm/metric/internal/LatencyStatAndMetric.java
index c66f2a1..ddaadb7 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/LatencyStatAndMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/LatencyStatAndMetric.java
@@ -31,37 +31,39 @@
//The current lat and count buckets are protected by a different lock
// from the other buckets. This is to reduce the lock contention
// When doing complex calculations. Never grab the instance object lock
- // while holding _currentLock to avoid deadlocks
- private final Object _currentLock = new byte[0];
+ // while holding currentLock to avoid deadlocks
+ private final Object currentLock = new byte[0];
//10 min values
- private final int _tmSize;
- private final long[] _tmLatBuckets;
- private final long[] _tmCountBuckets;
- private final long[] _tmTime;
+ private final int tmSize;
+ private final long[] tmLatBuckets;
+ private final long[] tmCountBuckets;
+ private final long[] tmTime;
//3 hour values
- private final int _thSize;
- private final long[] _thLatBuckets;
- private final long[] _thCountBuckets;
- private final long[] _thTime;
+ private final int thSize;
+ private final long[] thLatBuckets;
+ private final long[] thCountBuckets;
+ private final long[] thTime;
//1 day values
- private final int _odSize;
- private final long[] _odLatBuckets;
- private final long[] _odCountBuckets;
- private final long[] _odTime;
- private final TimerTask _task;
- private long _currentLatBucket;
- private long _currentCountBucket;
+ private final int odSize;
+ private final long[] odLatBuckets;
+ private final long[] odCountBuckets;
+ private final long[] odTime;
+ private final TimerTask task;
+ private long currentLatBucket;
+ private long currentCountBucket;
// All internal state except for the current buckets are
// protected using the Object Lock
- private long _bucketStart;
+ private long bucketStart;
//exact variable time, that is added to the current bucket
- private long _exactExtraLat;
- private long _exactExtraCount;
+ private long exactExtraLat;
+ private long exactExtraCount;
//all time
- private long _allTimeLat;
- private long _allTimeCount;
+ private long allTimeLat;
+ private long allTimeCount;
/**
+ * Constructor.
+ *
* @param numBuckets the number of buckets to divide the time periods into.
*/
public LatencyStatAndMetric(int numBuckets) {
@@ -69,7 +71,7 @@
}
/**
- * Constructor
+ * Constructor.
*
* @param numBuckets the number of buckets to divide the time periods into.
* @param startTime if positive the simulated time to start the from.
@@ -78,46 +80,46 @@
numBuckets = Math.max(numBuckets, 2);
//We want to capture the full time range, so the target size is as
// if we had one bucket less, then we do
- _tmSize = 10 * 60 * 1000 / (numBuckets - 1);
- _thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
- _odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
- if (_tmSize < 1 || _thSize < 1 || _odSize < 1) {
+ tmSize = 10 * 60 * 1000 / (numBuckets - 1);
+ thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
+ odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
+ if (tmSize < 1 || thSize < 1 || odSize < 1) {
throw new IllegalArgumentException("number of buckets is too large to be supported");
}
- _tmLatBuckets = new long[numBuckets];
- _tmCountBuckets = new long[numBuckets];
- _tmTime = new long[numBuckets];
- _thLatBuckets = new long[numBuckets];
- _thCountBuckets = new long[numBuckets];
- _thTime = new long[numBuckets];
- _odLatBuckets = new long[numBuckets];
- _odCountBuckets = new long[numBuckets];
- _odTime = new long[numBuckets];
- _allTimeLat = 0;
- _allTimeCount = 0;
- _exactExtraLat = 0;
- _exactExtraCount = 0;
+ tmLatBuckets = new long[numBuckets];
+ tmCountBuckets = new long[numBuckets];
+ tmTime = new long[numBuckets];
+ thLatBuckets = new long[numBuckets];
+ thCountBuckets = new long[numBuckets];
+ thTime = new long[numBuckets];
+ odLatBuckets = new long[numBuckets];
+ odCountBuckets = new long[numBuckets];
+ odTime = new long[numBuckets];
+ allTimeLat = 0;
+ allTimeCount = 0;
+ exactExtraLat = 0;
+ exactExtraCount = 0;
- _bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
- _currentLatBucket = 0;
- _currentCountBucket = 0;
+ bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
+ currentLatBucket = 0;
+ currentCountBucket = 0;
if (startTime < 0) {
- _task = new Fresher();
- MetricStatTimer._timer.scheduleAtFixedRate(_task, _tmSize, _tmSize);
+ task = new Fresher();
+ MetricStatTimer.timer.scheduleAtFixedRate(task, tmSize, tmSize);
} else {
- _task = null;
+ task = null;
}
}
/**
- * Record a specific latency
+ * Record a specific latency.
*
* @param latency what we are recording
*/
public void record(long latency) {
- synchronized (_currentLock) {
- _currentLatBucket += latency;
- _currentCountBucket++;
+ synchronized (currentLock) {
+ currentLatBucket += latency;
+ currentCountBucket++;
}
}
@@ -129,20 +131,22 @@
synchronized Object getValueAndReset(long now) {
long lat;
long count;
- synchronized (_currentLock) {
- lat = _currentLatBucket;
- count = _currentCountBucket;
- _currentLatBucket = 0;
- _currentCountBucket = 0;
+ synchronized (currentLock) {
+ lat = currentLatBucket;
+ count = currentCountBucket;
+ currentLatBucket = 0;
+ currentCountBucket = 0;
}
- long timeSpent = now - _bucketStart;
- long exactExtraCountSum = count + _exactExtraCount;
+ @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
+ long timeSpent = now - bucketStart;
+ long exactExtraCountSum = count + exactExtraCount;
+ @SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
double ret = Utils.zeroIfNaNOrInf(
- ((double) (lat + _exactExtraLat)) / exactExtraCountSum);
- _bucketStart = now;
- _exactExtraLat = 0;
- _exactExtraCount = 0;
+ ((double) (lat + exactExtraLat)) / exactExtraCountSum);
+ bucketStart = now;
+ exactExtraLat = 0;
+ exactExtraCount = 0;
rotateBuckets(lat, count, timeSpent);
return ret;
}
@@ -150,26 +154,26 @@
synchronized void rotateSched(long now) {
long lat;
long count;
- synchronized (_currentLock) {
- lat = _currentLatBucket;
- count = _currentCountBucket;
- _currentLatBucket = 0;
- _currentCountBucket = 0;
+ synchronized (currentLock) {
+ lat = currentLatBucket;
+ count = currentCountBucket;
+ currentLatBucket = 0;
+ currentCountBucket = 0;
}
- long timeSpent = now - _bucketStart;
- _exactExtraLat += lat;
- _exactExtraCount += count;
- _bucketStart = now;
+ exactExtraLat += lat;
+ exactExtraCount += count;
+ long timeSpent = now - bucketStart;
+ bucketStart = now;
rotateBuckets(lat, count, timeSpent);
}
synchronized void rotateBuckets(long lat, long count, long timeSpent) {
- rotate(lat, count, timeSpent, _tmSize, _tmTime, _tmLatBuckets, _tmCountBuckets);
- rotate(lat, count, timeSpent, _thSize, _thTime, _thLatBuckets, _thCountBuckets);
- rotate(lat, count, timeSpent, _odSize, _odTime, _odLatBuckets, _odCountBuckets);
- _allTimeLat += lat;
- _allTimeCount += count;
+ rotate(lat, count, timeSpent, tmSize, tmTime, tmLatBuckets, tmCountBuckets);
+ rotate(lat, count, timeSpent, thSize, thTime, thLatBuckets, thCountBuckets);
+ rotate(lat, count, timeSpent, odSize, odTime, odLatBuckets, odCountBuckets);
+ allTimeLat += lat;
+ allTimeCount += count;
}
private synchronized void rotate(long lat, long count, long timeSpent, long targetSize,
@@ -199,6 +203,7 @@
}
/**
+ * Get time latency average.
* @return a map of time window to average latency. Keys are "600" for last 10 mins "10800" for the last 3 hours "86400" for the last
* day ":all-time" for all time
*/
@@ -210,17 +215,17 @@
Map<String, Double> ret = new HashMap<>();
long lat;
long count;
- synchronized (_currentLock) {
- lat = _currentLatBucket;
- count = _currentCountBucket;
+ synchronized (currentLock) {
+ lat = currentLatBucket;
+ count = currentCountBucket;
}
- long timeSpent = now - _bucketStart;
- ret.put("600", readApproximateLatAvg(lat, count, timeSpent, _tmTime, _tmLatBuckets, _tmCountBuckets, 600 * 1000));
- ret.put("10800", readApproximateLatAvg(lat, count, timeSpent, _thTime, _thLatBuckets, _thCountBuckets, 10800 * 1000));
- ret.put("86400", readApproximateLatAvg(lat, count, timeSpent, _odTime, _odLatBuckets, _odCountBuckets, 86400 * 1000));
- long allTimeCountSum = count + _allTimeCount;
+ long timeSpent = now - bucketStart;
+ ret.put("600", readApproximateLatAvg(lat, count, timeSpent, tmTime, tmLatBuckets, tmCountBuckets, 600 * 1000));
+ ret.put("10800", readApproximateLatAvg(lat, count, timeSpent, thTime, thLatBuckets, thCountBuckets, 10800 * 1000));
+ ret.put("86400", readApproximateLatAvg(lat, count, timeSpent, odTime, odLatBuckets, odCountBuckets, 86400 * 1000));
+ long allTimeCountSum = count + allTimeCount;
ret.put(":all-time", Utils.zeroIfNaNOrInf(
- (double) lat + _allTimeLat) / allTimeCountSum);
+ (double) lat + allTimeLat) / allTimeCountSum);
return ret;
}
@@ -239,8 +244,8 @@
}
public void close() {
- if (_task != null) {
- _task.cancel();
+ if (task != null) {
+ task.cancel();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/MetricStatTimer.java b/storm-client/src/jvm/org/apache/storm/metric/internal/MetricStatTimer.java
index 4541a25..41b87a5 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/MetricStatTimer.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/MetricStatTimer.java
@@ -15,8 +15,8 @@
import java.util.Timer;
/**
- * Just holds a singleton metric/stat timer for use by metric/stat calculations
+ * Just holds a singleton metric/stat timer for use by metric/stat calculations.
*/
class MetricStatTimer {
- static Timer _timer = new Timer("metric/stat timer", true);
+ static Timer timer = new Timer("metric/stat timer", true);
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/MultiCountStatAndMetric.java b/storm-client/src/jvm/org/apache/storm/metric/internal/MultiCountStatAndMetric.java
index 1d44863..8e0b093 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/MultiCountStatAndMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/MultiCountStatAndMetric.java
@@ -22,24 +22,26 @@
* Acts as a MultiCount Metric, but keeps track of approximate counts for the last 10 mins, 3 hours, 1 day, and all time. for the same keys
*/
public class MultiCountStatAndMetric<T> implements IMetric {
- private final int _numBuckets;
- private ConcurrentHashMap<T, CountStatAndMetric> _counts = new ConcurrentHashMap<>();
+ private final int numBuckets;
+ private ConcurrentHashMap<T, CountStatAndMetric> counts = new ConcurrentHashMap<>();
/**
+ * Constructor.
+ *
* @param numBuckets the number of buckets to divide the time periods into.
*/
public MultiCountStatAndMetric(int numBuckets) {
- _numBuckets = numBuckets;
+ this.numBuckets = numBuckets;
}
CountStatAndMetric get(T key) {
- CountStatAndMetric c = _counts.get(key);
+ CountStatAndMetric c = counts.get(key);
if (c == null) {
synchronized (this) {
- c = _counts.get(key);
+ c = counts.get(key);
if (c == null) {
- c = new CountStatAndMetric(_numBuckets);
- _counts.put(key, c);
+ c = new CountStatAndMetric(numBuckets);
+ counts.put(key, c);
}
}
}
@@ -68,12 +70,12 @@
@Override
public Object getValueAndReset() {
Map<String, Long> ret = new HashMap<String, Long>();
- for (Map.Entry<T, CountStatAndMetric> entry : _counts.entrySet()) {
+ for (Map.Entry<T, CountStatAndMetric> entry : counts.entrySet()) {
String key = keyToString(entry.getKey());
//There could be collisions if keyToString returns only part of a result.
Long val = (Long) entry.getValue().getValueAndReset();
Long other = ret.get(key);
- val += other == null ? 0l : other;
+ val += other == null ? 0L : other;
ret.put(key, val);
}
return ret;
@@ -81,7 +83,7 @@
public Map<String, Map<T, Long>> getTimeCounts() {
Map<String, Map<T, Long>> ret = new HashMap<>();
- for (Map.Entry<T, CountStatAndMetric> entry : _counts.entrySet()) {
+ for (Map.Entry<T, CountStatAndMetric> entry : counts.entrySet()) {
T key = entry.getKey();
Map<String, Long> toFlip = entry.getValue().getTimeCounts();
for (Map.Entry<String, Long> subEntry : toFlip.entrySet()) {
@@ -98,7 +100,7 @@
}
public void close() {
- for (CountStatAndMetric cc : _counts.values()) {
+ for (CountStatAndMetric cc : counts.values()) {
cc.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/MultiLatencyStatAndMetric.java b/storm-client/src/jvm/org/apache/storm/metric/internal/MultiLatencyStatAndMetric.java
index e94bc62..eae373c 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/MultiLatencyStatAndMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/MultiLatencyStatAndMetric.java
@@ -23,24 +23,26 @@
* the same keys
*/
public class MultiLatencyStatAndMetric<T> implements IMetric {
- private final int _numBuckets;
- private ConcurrentHashMap<T, LatencyStatAndMetric> _lat = new ConcurrentHashMap<>();
+ private final int numBuckets;
+ private ConcurrentHashMap<T, LatencyStatAndMetric> lat = new ConcurrentHashMap<>();
/**
+ * Constructor.
+ *
* @param numBuckets the number of buckets to divide the time periods into.
*/
public MultiLatencyStatAndMetric(int numBuckets) {
- _numBuckets = numBuckets;
+ this.numBuckets = numBuckets;
}
LatencyStatAndMetric get(T key) {
- LatencyStatAndMetric c = _lat.get(key);
+ LatencyStatAndMetric c = lat.get(key);
if (c == null) {
synchronized (this) {
- c = _lat.get(key);
+ c = lat.get(key);
if (c == null) {
- c = new LatencyStatAndMetric(_numBuckets);
- _lat.put(key, c);
+ c = new LatencyStatAndMetric(numBuckets);
+ lat.put(key, c);
}
}
}
@@ -48,7 +50,7 @@
}
/**
- * Record a latency value
+ * Record a latency value.
*
* @param latency the measurement to record
*/
@@ -69,7 +71,7 @@
@Override
public Object getValueAndReset() {
Map<String, Double> ret = new HashMap<String, Double>();
- for (Map.Entry<T, LatencyStatAndMetric> entry : _lat.entrySet()) {
+ for (Map.Entry<T, LatencyStatAndMetric> entry : lat.entrySet()) {
String key = keyToString(entry.getKey());
Double val = (Double) entry.getValue().getValueAndReset();
ret.put(key, val);
@@ -79,7 +81,7 @@
public Map<String, Map<T, Double>> getTimeLatAvg() {
Map<String, Map<T, Double>> ret = new HashMap<>();
- for (Map.Entry<T, LatencyStatAndMetric> entry : _lat.entrySet()) {
+ for (Map.Entry<T, LatencyStatAndMetric> entry : lat.entrySet()) {
T key = entry.getKey();
Map<String, Double> toFlip = entry.getValue().getTimeLatAvg();
for (Map.Entry<String, Double> subEntry : toFlip.entrySet()) {
@@ -96,7 +98,7 @@
}
public void close() {
- for (LatencyStatAndMetric l : _lat.values()) {
+ for (LatencyStatAndMetric l : lat.values()) {
l.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/metric/internal/RateTracker.java b/storm-client/src/jvm/org/apache/storm/metric/internal/RateTracker.java
index d0bbc74..94f3433 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/internal/RateTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/internal/RateTracker.java
@@ -20,18 +20,20 @@
* This class is a utility to track the rate of something.
*/
public class RateTracker implements Closeable {
- private final int _bucketSizeMillis;
+ private final int bucketSizeMillis;
//Old Buckets and their length are only touched when rotating or gathering the metrics, which should not be that frequent
// As such all access to them should be protected by synchronizing with the RateTracker instance
- private final long[] _bucketTime;
- private final long[] _oldBuckets;
+ private final long[] bucketTime;
+ private final long[] oldBuckets;
- private final AtomicLong _bucketStart;
- private final AtomicLong _currentBucket;
+ private final AtomicLong bucketStart;
+ private final AtomicLong currentBucket;
- private final TimerTask _task;
+ private final TimerTask task;
/**
+ * Constructor.
+ *
* @param validTimeWindowInMils events that happened before validTimeWindowInMils are not considered when reporting the rate.
* @param numBuckets the number of time sildes to divide validTimeWindows. The more buckets, the smother the reported results
* will be.
@@ -41,7 +43,7 @@
}
/**
- * Constructor
+ * Constructor.
*
* @param validTimeWindowInMils events that happened before validTimeWindow are not considered when reporting the rate.
* @param numBuckets the number of time sildes to divide validTimeWindows. The more buckets, the smother the reported results
@@ -50,34 +52,35 @@
*/
RateTracker(int validTimeWindowInMils, int numBuckets, long startTime) {
numBuckets = Math.max(numBuckets, 1);
- _bucketSizeMillis = validTimeWindowInMils / numBuckets;
- if (_bucketSizeMillis < 1) {
+ bucketSizeMillis = validTimeWindowInMils / numBuckets;
+ if (bucketSizeMillis < 1) {
throw new IllegalArgumentException(
"validTimeWindowInMilis and numOfSildes cause each slide to have a window that is too small");
}
- _bucketTime = new long[numBuckets - 1];
- _oldBuckets = new long[numBuckets - 1];
+ bucketTime = new long[numBuckets - 1];
+ oldBuckets = new long[numBuckets - 1];
- _bucketStart = new AtomicLong(startTime >= 0 ? startTime : System.currentTimeMillis());
- _currentBucket = new AtomicLong(0);
+ bucketStart = new AtomicLong(startTime >= 0 ? startTime : System.currentTimeMillis());
+ currentBucket = new AtomicLong(0);
if (startTime < 0) {
- _task = new Fresher();
- MetricStatTimer._timer.scheduleAtFixedRate(_task, _bucketSizeMillis, _bucketSizeMillis);
+ task = new Fresher();
+ MetricStatTimer.timer.scheduleAtFixedRate(task, bucketSizeMillis, bucketSizeMillis);
} else {
- _task = null;
+ task = null;
}
}
/**
- * Notify the tracker upon new arrivals
+ * Notify the tracker upon new arrivals.
*
* @param count number of arrivals
*/
public void notify(long count) {
- _currentBucket.addAndGet(count);
+ currentBucket.addAndGet(count);
}
/**
+ * Get report rate.
* @return the approximate average rate per second.
*/
public synchronized double reportRate() {
@@ -85,11 +88,11 @@
}
synchronized double reportRate(long currentTime) {
- long duration = Math.max(1l, currentTime - _bucketStart.get());
- long events = _currentBucket.get();
- for (int i = 0; i < _oldBuckets.length; i++) {
- events += _oldBuckets[i];
- duration += _bucketTime[i];
+ long duration = Math.max(1L, currentTime - bucketStart.get());
+ long events = currentBucket.get();
+ for (int i = 0; i < oldBuckets.length; i++) {
+ events += oldBuckets[i];
+ duration += bucketTime[i];
}
return events * 1000.0 / duration;
@@ -97,8 +100,8 @@
@Override
public void close() {
- if (_task != null) {
- _task.cancel();
+ if (task != null) {
+ task.cancel();
}
}
@@ -108,7 +111,7 @@
* @param numToEclipse the number of rotations to perform.
*/
final void forceRotate(int numToEclipse, long interval) {
- long time = _bucketStart.get();
+ long time = bucketStart.get();
for (int i = 0; i < numToEclipse; i++) {
time += interval;
rotateBuckets(time);
@@ -116,15 +119,15 @@
}
private synchronized void rotateBuckets(long time) {
- long timeSpent = time - _bucketStart.getAndSet(time);
- long currentVal = _currentBucket.getAndSet(0);
- for (int i = 0; i < _oldBuckets.length; i++) {
- long tmpTime = _bucketTime[i];
- _bucketTime[i] = timeSpent;
+ long timeSpent = time - bucketStart.getAndSet(time);
+ long currentVal = currentBucket.getAndSet(0);
+ for (int i = 0; i < oldBuckets.length; i++) {
+ long tmpTime = bucketTime[i];
+ bucketTime[i] = timeSpent;
timeSpent = tmpTime;
- long cnt = _oldBuckets[i];
- _oldBuckets[i] = currentVal;
+ long cnt = oldBuckets[i];
+ oldBuckets[i] = currentVal;
currentVal = cnt;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/multilang/BoltMsg.java b/storm-client/src/jvm/org/apache/storm/multilang/BoltMsg.java
index 5153664..956780f 100644
--- a/storm-client/src/jvm/org/apache/storm/multilang/BoltMsg.java
+++ b/storm-client/src/jvm/org/apache/storm/multilang/BoltMsg.java
@@ -18,8 +18,9 @@
* BoltMsg is an object that represents the data sent from a shell component to a bolt process that implements a multi-language protocol. It
* is the union of all data types that a bolt can receive from Storm.
*
- * BoltMsgs are objects sent to the ISerializer interface, for serialization according to the wire protocol implemented by the serializer.
- * The BoltMsg class allows for a decoupling between the serialized representation of the data and the data itself.
+ * <p>BoltMsgs are objects sent to the ISerializer interface, for serialization according to the wire protocol
+ * implemented by the serializer. The BoltMsg class allows for a decoupling between the serialized representation of the
+ * data and the data itself.
*/
public class BoltMsg {
private String id;
diff --git a/storm-client/src/jvm/org/apache/storm/multilang/ISerializer.java b/storm-client/src/jvm/org/apache/storm/multilang/ISerializer.java
index 1ac87a8..bc7c3ac 100644
--- a/storm-client/src/jvm/org/apache/storm/multilang/ISerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/multilang/ISerializer.java
@@ -27,7 +27,7 @@
public interface ISerializer extends Serializable {
/**
- * This method sets the input and output streams of the serializer
+ * This method sets the input and output streams of the serializer.
*
* @param processIn output stream to non-JVM component
* @param processOut input stream from non-JVM component
@@ -45,28 +45,28 @@
NoOutputException;
/**
- * This method receives a shell message from the non-JVM process
+ * This method receives a shell message from the non-JVM process.
*
* @return shell message
*/
ShellMsg readShellMsg() throws IOException, NoOutputException;
/**
- * This method sends a bolt message to a non-JVM bolt process
+ * This method sends a bolt message to a non-JVM bolt process.
*
* @param msg bolt message
*/
void writeBoltMsg(BoltMsg msg) throws IOException;
/**
- * This method sends a spout message to a non-JVM spout process
+ * This method sends a spout message to a non-JVM spout process.
*
* @param msg spout message
*/
void writeSpoutMsg(SpoutMsg msg) throws IOException;
/**
- * This method sends a list of task IDs to a non-JVM bolt process
+ * This method sends a list of task IDs to a non-JVM bolt process.
*
* @param taskIds list of task IDs
*/
diff --git a/storm-client/src/jvm/org/apache/storm/multilang/JsonSerializer.java b/storm-client/src/jvm/org/apache/storm/multilang/JsonSerializer.java
index 6b44d19..674466e 100644
--- a/storm-client/src/jvm/org/apache/storm/multilang/JsonSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/multilang/JsonSerializer.java
@@ -123,8 +123,8 @@
shellMsg.setTask(0);
}
- Object need_task_ids = msg.get("need_task_ids");
- if (need_task_ids == null || ((Boolean) need_task_ids).booleanValue()) {
+ Object needTaskIds = msg.get("need_task_ids");
+ if (needTaskIds == null || ((Boolean) needTaskIds).booleanValue()) {
shellMsg.setNeedTaskIds(true);
} else {
shellMsg.setNeedTaskIds(false);
diff --git a/storm-client/src/jvm/org/apache/storm/multilang/ShellMsg.java b/storm-client/src/jvm/org/apache/storm/multilang/ShellMsg.java
index 881043a..0695d86 100644
--- a/storm-client/src/jvm/org/apache/storm/multilang/ShellMsg.java
+++ b/storm-client/src/jvm/org/apache/storm/multilang/ShellMsg.java
@@ -19,7 +19,7 @@
* ShellMsg is an object that represents the data sent to a shell component from a process that implements a multi-language protocol. It is
* the union of all data types that a component can send to Storm.
*
- * ShellMsgs are objects received from the ISerializer interface, after the serializer has deserialized the data from the underlying wire
+ * <p>ShellMsgs are objects received from the ISerializer interface, after the serializer has deserialized the data from the underlying wire
* protocol. The ShellMsg class allows for a decoupling between the serialized representation of the data and the data itself.
*/
public class ShellMsg {
@@ -141,19 +141,19 @@
@Override
public String toString() {
- return "ShellMsg{" +
- "command='" + command + '\'' +
- ", id=" + id +
- ", anchors=" + anchors +
- ", stream='" + stream + '\'' +
- ", task=" + task +
- ", msg='" + msg + '\'' +
- ", tuple=" + tuple +
- ", needTaskIds=" + needTaskIds +
- ", metricName='" + metricName + '\'' +
- ", metricParams=" + metricParams +
- ", logLevel=" + logLevel +
- '}';
+ return "ShellMsg{"
+ + "command='" + command + '\''
+ + ", id=" + id
+ + ", anchors=" + anchors
+ + ", stream='" + stream + '\''
+ + ", task=" + task
+ + ", msg='" + msg + '\''
+ + ", tuple=" + tuple
+ + ", needTaskIds=" + needTaskIds
+ + ", metricName='" + metricName + '\''
+ + ", metricParams=" + metricParams
+ + ", logLevel=" + logLevel
+ + '}';
}
//logLevel
diff --git a/storm-client/src/jvm/org/apache/storm/multilang/SpoutMsg.java b/storm-client/src/jvm/org/apache/storm/multilang/SpoutMsg.java
index 13eebae..870ed03 100644
--- a/storm-client/src/jvm/org/apache/storm/multilang/SpoutMsg.java
+++ b/storm-client/src/jvm/org/apache/storm/multilang/SpoutMsg.java
@@ -16,7 +16,7 @@
* SpoutMsg is an object that represents the data sent from a shell spout to a process that implements a multi-language spout. The SpoutMsg
* is used to send a "next", "ack" or "fail" message to a spout.
*
- * Spout messages are objects sent to the ISerializer interface, for serialization according to the wire protocol implemented by the
+ * <p>Spout messages are objects sent to the ISerializer interface, for serialization according to the wire protocol implemented by the
* serializer. The SpoutMsg class allows for a decoupling between the serialized representation of the data and the data itself.
*/
public class SpoutMsg {
diff --git a/storm-client/src/jvm/org/apache/storm/networktopography/AbstractDNSToSwitchMapping.java b/storm-client/src/jvm/org/apache/storm/networktopography/AbstractDNSToSwitchMapping.java
index 1be30ca..480e32e 100644
--- a/storm-client/src/jvm/org/apache/storm/networktopography/AbstractDNSToSwitchMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/networktopography/AbstractDNSToSwitchMapping.java
@@ -19,14 +19,16 @@
/**
* This is a base class for DNS to Switch mappings.
*
- * It is not mandatory to derive {@link DNSToSwitchMapping} implementations from it, but it is strongly recommended, as it makes it easy for
- * the developers to add new methods to this base class that are automatically picked up by all implementations.
+ * <p>It is not mandatory to derive {@link DNSToSwitchMapping} implementations from it, but it is strongly recommended,
+ * as it makes it easy for the developers to add new methods to this base class that are automatically picked up by all
+ * implementations.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public abstract class AbstractDNSToSwitchMapping
implements DNSToSwitchMapping {
/**
- * Create an unconfigured instance
+ * Create an unconfigured instance.
*/
protected AbstractDNSToSwitchMapping() {
}
@@ -42,7 +44,7 @@
}
/**
- * Get a copy of the map (for diagnostics)
+ * Get a copy of the map (for diagnostics).
*
* @return a clone of the map or null for none known
*/
diff --git a/storm-client/src/jvm/org/apache/storm/networktopography/DNSToSwitchMapping.java b/storm-client/src/jvm/org/apache/storm/networktopography/DNSToSwitchMapping.java
index ef60a9c..e2cf14b 100644
--- a/storm-client/src/jvm/org/apache/storm/networktopography/DNSToSwitchMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/networktopography/DNSToSwitchMapping.java
@@ -18,20 +18,22 @@
/**
* An interface that must be implemented to allow pluggable DNS-name/IP-address to RackID resolvers.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface DNSToSwitchMapping {
- public final static String DEFAULT_RACK = "/default-rack";
+ String DEFAULT_RACK = "/default-rack";
/**
- * Resolves a list of DNS-names/IP-address and returns back a map of DNS-name->switch information ( network paths). Consider an element
- * in the argument list - x.y.com. The switch information that is returned must be a network path of the form /foo/rack, where / is the
- * root, and 'foo' is the switch where 'rack' is connected. Note the hostname/ip-address is not part of the returned path. The network
- * topology of the cluster would determine the number of components in the network path.
+ * Resolves a list of DNS-names/IP-address and returns back a map of DNS-name->switch information ( network paths).
+ * Consider an element in the argument list - x.y.com. The switch information that is returned must be a network
+ * path of the form /foo/rack, where / is the root, and 'foo' is the switch where 'rack' is connected. Note the
+ * hostname/ip-address is not part of the returned path. The network topology of the cluster would determine the
+ * number of components in the network path.
*
- * If a name cannot be resolved to a rack, the implementation should return {DEFAULT_RACK}. This is what the bundled implementations do,
- * though it is not a formal requirement
+ * <p>If a name cannot be resolved to a rack, the implementation should return {DEFAULT_RACK}. This is what the
+ * bundled implementations do, though it is not a formal requirement.
*
* @param names the list of hosts to resolve (can be empty)
* @return Map of hosts to resolved network paths. If <i>names</i> is empty, then return empty Map
*/
- public Map<String, String> resolve(List<String> names);
+ Map<String, String> resolve(List<String> names);
}
diff --git a/storm-client/src/jvm/org/apache/storm/networktopography/DefaultRackDNSToSwitchMapping.java b/storm-client/src/jvm/org/apache/storm/networktopography/DefaultRackDNSToSwitchMapping.java
index 3262c1f..9453513 100644
--- a/storm-client/src/jvm/org/apache/storm/networktopography/DefaultRackDNSToSwitchMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/networktopography/DefaultRackDNSToSwitchMapping.java
@@ -20,6 +20,7 @@
/**
* This class implements the {@link DNSToSwitchMapping} interface It returns the DEFAULT_RACK for every host.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public final class DefaultRackDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
private Map<String, String> mappingCache = new ConcurrentHashMap<>();
diff --git a/storm-client/src/jvm/org/apache/storm/nimbus/NimbusInfo.java b/storm-client/src/jvm/org/apache/storm/nimbus/NimbusInfo.java
index d3c5f2b..29400e1 100644
--- a/storm-client/src/jvm/org/apache/storm/nimbus/NimbusInfo.java
+++ b/storm-client/src/jvm/org/apache/storm/nimbus/NimbusInfo.java
@@ -119,10 +119,10 @@
@Override
public String toString() {
- return "NimbusInfo{" +
- "host='" + host + '\'' +
- ", port=" + port +
- ", isLeader=" + isLeader +
- '}';
+ return "NimbusInfo{"
+ + "host='" + host + '\''
+ + ", port=" + port
+ + ", isLeader=" + isLeader
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java b/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index 641e326..b77e003 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@ -45,14 +45,14 @@
private static Timer timer = new Timer(true);
private final Bootstrap bootstrap;
private final EventLoopGroup workerEventLoopGroup;
- private String client_name;
+ private String clientName;
private String secret;
private AtomicBoolean ready;
private AtomicBoolean shutdown;
private AtomicReference<Channel> channelRef;
- private InetSocketAddress remote_addr;
+ private InetSocketAddress remoteAddr;
private int maxPending = 100;
- private HBMessage messages[];
+ private HBMessage[] messages;
private LinkedBlockingQueue<Integer> availableMessageSlots;
private ThriftNettyClientCodec.AuthMethod authMethod;
private static final int maxRetries = 10;
@@ -63,21 +63,19 @@
public PacemakerClient(Map<String, Object> config, String host) {
this.host = host;
- int port = (int) config.get(Config.PACEMAKER_PORT);
- client_name = (String) config.get(Config.TOPOLOGY_NAME);
- if (client_name == null) {
- client_name = "pacemaker-client";
+ clientName = (String) config.get(Config.TOPOLOGY_NAME);
+ if (clientName == null) {
+ clientName = "pacemaker-client";
}
- int maxWorkers = (int)config.get(Config.PACEMAKER_CLIENT_MAX_THREADS);
String auth = (String) config.get(Config.PACEMAKER_AUTH_METHOD);
switch (auth) {
case "DIGEST":
- Configuration login_conf = ClientAuthUtils.getConfiguration(config);
+ Configuration loginConf = ClientAuthUtils.getConfiguration(config);
authMethod = ThriftNettyClientCodec.AuthMethod.DIGEST;
- secret = ClientAuthUtils.makeDigestPayload(login_conf, ClientAuthUtils.LOGIN_CONTEXT_PACEMAKER_DIGEST);
+ secret = ClientAuthUtils.makeDigestPayload(loginConf, ClientAuthUtils.LOGIN_CONTEXT_PACEMAKER_DIGEST);
if (secret == null) {
LOG.error("Can't start pacemaker server without digest secret.");
throw new RuntimeException("Can't start pacemaker server without digest secret.");
@@ -106,6 +104,7 @@
ThreadFactory workerFactory = new NettyRenameThreadFactory(this.host + "-pm");
// 0 means DEFAULT_EVENT_LOOP_THREADS
// https://github.com/netty/netty/blob/netty-4.1.24.Final/transport/src/main/java/io/netty/channel/MultithreadEventLoopGroup.java#L40
+ int maxWorkers = (int)config.get(Config.PACEMAKER_CLIENT_MAX_THREADS);
this.workerEventLoopGroup = new NioEventLoopGroup(maxWorkers > 0 ? maxWorkers : 0, workerFactory);
int thriftMessageMaxSize = (Integer) config.get(Config.PACEMAKER_THRIFT_MESSAGE_SIZE_MAX);
bootstrap = new Bootstrap()
@@ -118,8 +117,9 @@
.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
.handler(new ThriftNettyClientCodec(this, config, authMethod, host, thriftMessageMaxSize));
- remote_addr = new InetSocketAddress(host, port);
- bootstrap.connect(remote_addr);
+ int port = (int) config.get(Config.PACEMAKER_PORT);
+ remoteAddr = new InetSocketAddress(host, port);
+ bootstrap.connect(remoteAddr);
}
private void setupMessaging() {
@@ -147,7 +147,7 @@
@Override
public String name() {
- return client_name;
+ return clientName;
}
@Override
@@ -210,19 +210,19 @@
}
public void gotMessage(HBMessage m) {
- int message_id = m.get_message_id();
- if (message_id >= 0 && message_id < maxPending) {
+ int messageId = m.get_message_id();
+ if (messageId >= 0 && messageId < maxPending) {
LOG.debug("Pacemaker client got message: {}", m.toString());
- HBMessage request = messages[message_id];
+ HBMessage request = messages[messageId];
if (request == null) {
- LOG.debug("No message for slot: {}", Integer.toString(message_id));
+ LOG.debug("No message for slot: {}", Integer.toString(messageId));
} else {
synchronized (request) {
- messages[message_id] = m;
+ messages[messageId] = m;
request.notifyAll();
- availableMessageSlots.add(message_id);
+ availableMessageSlots.add(messageId);
}
}
} else {
@@ -246,7 +246,7 @@
LOG.info("reconnecting to {}", host);
close_channel();
if (!shutdown.get()) {
- bootstrap.connect(remote_addr);
+ bootstrap.connect(remoteAddr);
}
}
@@ -258,7 +258,7 @@
private synchronized void close_channel() {
if (channelRef.get() != null) {
channelRef.get().close();
- LOG.debug("channel {} closed", remote_addr);
+ LOG.debug("channel {} closed", remoteAddr);
channelRef.set(null);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java
index c88ecf2..ce104ed 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java
@@ -63,7 +63,7 @@
return;
}
- byte serialized[] = new byte[thriftLen];
+ byte[] serialized = new byte[thriftLen];
buf.readBytes(serialized, 0, thriftLen);
HBMessage m = (HBMessage) Utils.thriftDeserialize(HBMessage.class, serialized);
diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java
index 349803f..659f051 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java
@@ -34,19 +34,19 @@
.getLogger(ThriftEncoder.class);
private HBMessage encodeNettySerializable(ByteBufAllocator alloc,
- INettySerializable netty_message, HBServerMessageType mType) {
+ INettySerializable nettyMessage, HBServerMessageType serverMessageType) {
- HBMessageData message_data = new HBMessageData();
+ HBMessageData messageData = new HBMessageData();
HBMessage m = new HBMessage();
- byte[] messageBuffer = new byte[netty_message.encodeLength()];
+ byte[] messageBuffer = new byte[nettyMessage.encodeLength()];
ByteBuf wrappedBuffer = Unpooled.wrappedBuffer(messageBuffer);
try {
wrappedBuffer.resetWriterIndex();
- netty_message.write(wrappedBuffer);
+ nettyMessage.write(wrappedBuffer);
- message_data.set_message_blob(messageBuffer);
- m.set_type(mType);
- m.set_data(message_data);
+ messageData.set_message_blob(messageBuffer);
+ m.set_type(serverMessageType);
+ m.set_data(messageData);
return m;
} finally {
wrappedBuffer.release();
@@ -61,7 +61,7 @@
LOG.debug("Trying to encode: " + msg.getClass().toString() + " : " + msg.toString());
- HBMessage m;
+ HBMessage message;
ByteBufAllocator alloc = channelHandlerContext.alloc();
if (msg instanceof INettySerializable) {
INettySerializable nettyMsg = (INettySerializable) msg;
@@ -75,13 +75,13 @@
LOG.error("Didn't recognise INettySerializable: " + nettyMsg.toString());
throw new RuntimeException("Unrecognized INettySerializable.");
}
- m = encodeNettySerializable(alloc, nettyMsg, type);
+ message = encodeNettySerializable(alloc, nettyMsg, type);
} else {
- m = (HBMessage) msg;
+ message = (HBMessage) msg;
}
try {
- byte serialized[] = Utils.thriftSerialize(m);
+ byte[] serialized = Utils.thriftSerialize(message);
ByteBuf ret = alloc.ioBuffer(serialized.length + 4);
ret.writeInt(serialized.length);
diff --git a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
index 00e1bc9..8b9b0a2 100644
--- a/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
+++ b/storm-client/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
@@ -38,9 +38,9 @@
private final Map<String, Object> topoConf;
private final String host;
- public ThriftNettyClientCodec(PacemakerClient pacemaker_client, Map<String, Object> topoConf,
+ public ThriftNettyClientCodec(PacemakerClient pacemakerClient, Map<String, Object> topoConf,
AuthMethod authMethod, String host, int thriftMessageMaxSizeBytes) {
- client = pacemaker_client;
+ client = pacemakerClient;
this.authMethod = authMethod;
this.topoConf = topoConf;
this.host = host;
@@ -50,32 +50,32 @@
@Override
protected void initChannel(Channel ch) throws Exception {
ChannelPipeline pipeline = ch.pipeline();
- pipeline.addLast("encoder", new ThriftEncoder());
- pipeline.addLast("decoder", new ThriftDecoder(thriftMessageMaxSize));
+ pipeline.addLast("encoder", new ThriftEncoder());
+ pipeline.addLast("decoder", new ThriftDecoder(thriftMessageMaxSize));
- if (authMethod == AuthMethod.KERBEROS) {
- try {
- LOG.debug("Adding KerberosSaslClientHandler to pacemaker client pipeline.");
- pipeline.addLast(KERBEROS_HANDLER,
- new KerberosSaslClientHandler(client,
- topoConf,
- ClientAuthUtils.LOGIN_CONTEXT_PACEMAKER_CLIENT,
- host));
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- } else if (authMethod == AuthMethod.DIGEST) {
- try {
- LOG.debug("Adding SaslStormClientHandler to pacemaker client pipeline.");
- pipeline.addLast(SASL_HANDLER, new SaslStormClientHandler(client));
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- } else {
+ if (authMethod == AuthMethod.KERBEROS) {
+ try {
+ LOG.debug("Adding KerberosSaslClientHandler to pacemaker client pipeline.");
+ pipeline.addLast(KERBEROS_HANDLER,
+ new KerberosSaslClientHandler(client,
+ topoConf,
+ ClientAuthUtils.LOGIN_CONTEXT_PACEMAKER_CLIENT,
+ host));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ } else if (authMethod == AuthMethod.DIGEST) {
+ try {
+ LOG.debug("Adding SaslStormClientHandler to pacemaker client pipeline.");
+ pipeline.addLast(SASL_HANDLER, new SaslStormClientHandler(client));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ } else {
client.channelReady(ch);
- }
+ }
- pipeline.addLast("PacemakerClientHandler", new PacemakerClientHandler(client));
+ pipeline.addLast("PacemakerClientHandler", new PacemakerClientHandler(client));
}
public enum AuthMethod {
diff --git a/storm-client/src/jvm/org/apache/storm/policy/IWaitStrategy.java b/storm-client/src/jvm/org/apache/storm/policy/IWaitStrategy.java
index 160b6c8..1934a02 100644
--- a/storm-client/src/jvm/org/apache/storm/policy/IWaitStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/policy/IWaitStrategy.java
@@ -27,17 +27,16 @@
static IWaitStrategy createBackPressureWaitStrategy(Map<String, Object> topologyConf) {
IWaitStrategy producerWaitStrategy =
ReflectionUtils.newInstance((String) topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_STRATEGY));
- producerWaitStrategy.prepare(topologyConf, WAIT_SITUATION.BACK_PRESSURE_WAIT);
+ producerWaitStrategy.prepare(topologyConf, WaitSituation.BACK_PRESSURE_WAIT);
return producerWaitStrategy;
}
- void prepare(Map<String, Object> conf, WAIT_SITUATION waitSituation);
+ void prepare(Map<String, Object> conf, WaitSituation waitSituation);
/**
- * Implementations of this method should be thread-safe (preferably no side-effects and lock-free)
- * <p>
- * Supports static or dynamic backoff. Dynamic backoff relies on idleCounter to estimate how long caller has been idling.
- * <p>
+ * Implementations of this method should be thread-safe (preferably no side-effects and lock-free).
+ *
+ * <p>Supports static or dynamic backoff. Dynamic backoff relies on idleCounter to estimate how long caller has been idling.
* <pre>
* <code>
* int idleCounter = 0;
@@ -54,7 +53,9 @@
*/
int idle(int idleCounter) throws InterruptedException;
- enum WAIT_SITUATION {SPOUT_WAIT, BOLT_WAIT, BACK_PRESSURE_WAIT}
-
-
+ enum WaitSituation {
+ SPOUT_WAIT,
+ BOLT_WAIT,
+ BACK_PRESSURE_WAIT
+ }
}
diff --git a/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyPark.java b/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyPark.java
index c2e612c..702be12 100644
--- a/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyPark.java
+++ b/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyPark.java
@@ -35,12 +35,12 @@
}
@Override
- public void prepare(Map<String, Object> conf, WAIT_SITUATION waitSituation) {
- if (waitSituation == WAIT_SITUATION.SPOUT_WAIT) {
+ public void prepare(Map<String, Object> conf, WaitSituation waitSituation) {
+ if (waitSituation == WaitSituation.SPOUT_WAIT) {
parkTimeNanoSec = 1_000 * ObjectReader.getLong(conf.get(Config.TOPOLOGY_SPOUT_WAIT_PARK_MICROSEC));
- } else if (waitSituation == WAIT_SITUATION.BOLT_WAIT) {
+ } else if (waitSituation == WaitSituation.BOLT_WAIT) {
parkTimeNanoSec = 1_000 * ObjectReader.getLong(conf.get(Config.TOPOLOGY_BOLT_WAIT_PARK_MICROSEC));
- } else if (waitSituation == WAIT_SITUATION.BACK_PRESSURE_WAIT) {
+ } else if (waitSituation == WaitSituation.BACK_PRESSURE_WAIT) {
parkTimeNanoSec = 1_000 * ObjectReader.getLong(conf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_PARK_MICROSEC));
} else {
throw new IllegalArgumentException("Unknown wait situation : " + waitSituation);
diff --git a/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyProgressive.java b/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyProgressive.java
index 5893b36..a721db7 100644
--- a/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyProgressive.java
+++ b/storm-client/src/jvm/org/apache/storm/policy/WaitStrategyProgressive.java
@@ -24,15 +24,14 @@
import org.apache.storm.utils.ObjectReader;
/**
- * A Progressive Wait Strategy
- * <p> Has three levels of idling. Stays in each level for a configured number of iterations before entering the next level.
+ * A Progressive Wait Strategy.
+ *
+ * <p>Has three levels of idling. Stays in each level for a configured number of iterations before entering the next level.
* Level 1 - No idling. Returns immediately. Stays in this level for `level1Count` iterations. Level 2 - Calls LockSupport.parkNanos(1).
* Stays in this level for `level2Count` iterations Level 3 - Calls Thread.sleep(). Stays in this level until wait situation changes.
*
- * <p>
- * The initial spin can be useful to prevent downstream bolt from repeatedly sleeping/parking when the upstream component is a bit
+ * <p>The initial spin can be useful to prevent downstream bolt from repeatedly sleeping/parking when the upstream component is a bit
* relatively slower. Allows downstream bolt can enter deeper wait states only if the traffic to it appears to have reduced.
- * <p>
*/
public class WaitStrategyProgressive implements IWaitStrategy {
private int level1Count;
@@ -40,16 +39,16 @@
private long level3SleepMs;
@Override
- public void prepare(Map<String, Object> conf, WAIT_SITUATION waitSituation) {
- if (waitSituation == WAIT_SITUATION.SPOUT_WAIT) {
+ public void prepare(Map<String, Object> conf, WaitSituation waitSituation) {
+ if (waitSituation == WaitSituation.SPOUT_WAIT) {
level1Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL1_COUNT));
level2Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL2_COUNT));
level3SleepMs = ObjectReader.getLong(conf.get(Config.TOPOLOGY_SPOUT_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS));
- } else if (waitSituation == WAIT_SITUATION.BOLT_WAIT) {
+ } else if (waitSituation == WaitSituation.BOLT_WAIT) {
level1Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL1_COUNT));
level2Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL2_COUNT));
level3SleepMs = ObjectReader.getLong(conf.get(Config.TOPOLOGY_BOLT_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS));
- } else if (waitSituation == WAIT_SITUATION.BACK_PRESSURE_WAIT) {
+ } else if (waitSituation == WaitSituation.BACK_PRESSURE_WAIT) {
level1Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL1_COUNT));
level2Count = ObjectReader.getInt(conf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL2_COUNT));
level3SleepMs = ObjectReader.getLong(conf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_PROGRESSIVE_LEVEL3_SLEEP_MILLIS));
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/AutoSSL.java b/storm-client/src/jvm/org/apache/storm/security/auth/AutoSSL.java
index 73c4b3b..ee90676 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/AutoSSL.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/AutoSSL.java
@@ -31,8 +31,9 @@
* credentials map. The key in the credentials map is the name of the file. On the worker side it uses the filenames from the
* ssl.credential.files config to lookup the keys in the credentials map and decodes it and writes it back out as a file.
*
- * User is responsible for referencing them from the topology code as ./<filename>.
+ * <p>User is responsible for referencing them from the topology code as {@code filename}.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AutoSSL implements IAutoCredentials {
public static final String SSL_FILES_CONF = "ssl.credential.files";
private static final Logger LOG = LoggerFactory.getLogger(AutoSSL.class);
@@ -41,6 +42,7 @@
// Adds the serialized and base64 file to the credentials map as a string with the filename as
// the key.
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static void serializeSSLFile(String readFile, Map<String, String> credentials) {
try (FileInputStream in = new FileInputStream(readFile)) {
LOG.debug("serializing ssl file: {}", readFile);
@@ -60,15 +62,16 @@
}
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static void deserializeSSLFile(String credsKey, String directory,
Map<String, String> credentials) {
try {
LOG.debug("deserializing ssl file with key: {}", credsKey);
String resultStr = null;
- if (credentials != null &&
- credentials.containsKey(credsKey) &&
- credentials.get(credsKey) != null) {
+ if (credentials != null
+ && credentials.containsKey(credsKey)
+ && credentials.get(credsKey) != null) {
resultStr = credentials.get(credsKey);
}
if (resultStr != null) {
@@ -90,16 +93,18 @@
}
@VisibleForTesting
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
protected String getSSLWriteDirFromConf(Map<String, Object> conf) {
return "./";
}
@VisibleForTesting
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
Collection<String> getSSLFilesFromConf(Map<String, Object> conf) {
Object sslConf = conf.get(SSL_FILES_CONF);
if (sslConf == null) {
- LOG.info("No ssl files requested, if you want to use SSL please set {} to the " +
- "list of files", SSL_FILES_CONF);
+ LOG.info("No ssl files requested, if you want to use SSL please set {} to the list of files",
+ SSL_FILES_CONF);
return null;
}
Collection<String> sslFiles = null;
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java b/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java
index f2e2f6b..9fb0e4b 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java
@@ -61,35 +61,34 @@
private static final String PASSWORD = "password";
/**
- * Construct a JAAS configuration object per storm configuration file
+ * Construct a JAAS configuration object per storm configuration file.
*
* @param topoConf Storm configuration
* @return JAAS configuration object
*/
public static Configuration getConfiguration(Map<String, Object> topoConf) {
- Configuration login_conf = null;
+ Configuration loginConf = null;
//find login file configuration from Storm configuration
String loginConfigurationFile = (String) topoConf.get("java.security.auth.login.config");
if ((loginConfigurationFile != null) && (loginConfigurationFile.length() > 0)) {
- File config_file = new File(loginConfigurationFile);
- if (!config_file.canRead()) {
- throw new RuntimeException("File " + loginConfigurationFile +
- " cannot be read.");
+ File configFile = new File(loginConfigurationFile);
+ if (!configFile.canRead()) {
+ throw new RuntimeException("File " + loginConfigurationFile + " cannot be read.");
}
try {
- URI config_uri = config_file.toURI();
- login_conf = Configuration.getInstance("JavaLoginConfig", new URIParameter(config_uri));
+ URI configUri = configFile.toURI();
+ loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configUri));
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
- return login_conf;
+ return loginConf;
}
/**
- * Get configurations for a section
+ * Get configurations for a section.
*
* @param configuration The config to pull the key/value pairs out of.
* @param section The app configuration entry name to get stuff from.
@@ -101,7 +100,7 @@
return null;
}
- AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section);
+ AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(section);
if (configurationEntries == null) {
String errorMessage = "Could not find a '" + section + "' entry in this configuration.";
throw new IOException(errorMessage);
@@ -137,7 +136,7 @@
}
/**
- * Pull a the value given section and key from Configuration
+ * Pull a the value given section and key from Configuration.
*
* @param configuration The config to pull the key/value pairs out of.
* @param section The app configuration entry name to get stuff from.
@@ -161,7 +160,7 @@
}
/**
- * Construct a principal to local plugin
+ * Construct a principal to local plugin.
*
* @param topoConf storm configuration
* @return the plugin
@@ -169,11 +168,11 @@
public static IPrincipalToLocal getPrincipalToLocalPlugin(Map<String, Object> topoConf) {
IPrincipalToLocal ptol = null;
try {
- String ptol_klassName = (String) topoConf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
- if (ptol_klassName == null) {
+ String ptolClassname = (String) topoConf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
+ if (ptolClassname == null) {
LOG.warn("No principal to local given {}", Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
} else {
- ptol = ReflectionUtils.newInstance(ptol_klassName);
+ ptol = ReflectionUtils.newInstance(ptolClassname);
//TODO this can only ever be null if someone is doing something odd with mocking
// We should really fix the mocking and remove this
if (ptol != null) {
@@ -187,7 +186,7 @@
}
/**
- * Construct a group mapping service provider plugin
+ * Construct a group mapping service provider plugin.
*
* @param conf daemon configuration
* @return the plugin
@@ -195,11 +194,11 @@
public static IGroupMappingServiceProvider getGroupMappingServiceProviderPlugin(Map<String, Object> conf) {
IGroupMappingServiceProvider gmsp = null;
try {
- String gmsp_klassName = (String) conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN);
- if (gmsp_klassName == null) {
+ String gmspClassName = (String) conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN);
+ if (gmspClassName == null) {
LOG.warn("No group mapper given {}", Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN);
} else {
- gmsp = ReflectionUtils.newInstance(gmsp_klassName);
+ gmsp = ReflectionUtils.newInstance(gmspClassName);
if (gmsp != null) {
gmsp.prepare(conf);
}
@@ -281,7 +280,7 @@
}
/**
- * Get the key used to store a WorkerToken in the credentials map
+ * Get the key used to store a WorkerToken in the credentials map.
*
* @param type the type of service to get.
* @return the key as a String.
@@ -455,24 +454,24 @@
}
/**
- * Construct a transport plugin per storm configuration
+ * Construct a transport plugin per storm configuration.
*/
- public static ITransportPlugin getTransportPlugin(ThriftConnectionType type, Map<String, Object> topoConf, Configuration login_conf) {
+ public static ITransportPlugin getTransportPlugin(ThriftConnectionType type, Map<String, Object> topoConf, Configuration loginConf) {
try {
- String transport_plugin_klassName = type.getTransportPlugin(topoConf);
- ITransportPlugin transportPlugin = ReflectionUtils.newInstance(transport_plugin_klassName);
- transportPlugin.prepare(type, topoConf, login_conf);
+ String transportPluginClassName = type.getTransportPlugin(topoConf);
+ ITransportPlugin transportPlugin = ReflectionUtils.newInstance(transportPluginClassName);
+ transportPlugin.prepare(type, topoConf, loginConf);
return transportPlugin;
} catch (Exception e) {
throw new RuntimeException(e);
}
}
- public static String makeDigestPayload(Configuration login_config, String config_section) {
+ public static String makeDigestPayload(Configuration loginConfig, String configSection) {
String username = null;
String password = null;
try {
- Map<String, ?> results = ClientAuthUtils.pullConfig(login_config, config_section);
+ Map<String, ?> results = ClientAuthUtils.pullConfig(loginConfig, configSection);
username = (String) results.get(USERNAME);
password = (String) results.get(PASSWORD);
} catch (Exception e) {
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/DefaultPrincipalToLocal.java b/storm-client/src/jvm/org/apache/storm/security/auth/DefaultPrincipalToLocal.java
index 3e95613..40cf3a5 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/DefaultPrincipalToLocal.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/DefaultPrincipalToLocal.java
@@ -19,7 +19,7 @@
*/
public class DefaultPrincipalToLocal implements IPrincipalToLocal {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*/
@Override
public void prepare(Map<String, Object> topoConf) {
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/FixedGroupsMapping.java b/storm-client/src/jvm/org/apache/storm/security/auth/FixedGroupsMapping.java
index cf8721c..7456cbd 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/FixedGroupsMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/FixedGroupsMapping.java
@@ -29,13 +29,13 @@
public Map<String, Set<String>> cachedGroups = new HashMap<String, Set<String>>();
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
- * @param storm_conf Storm configuration
+ * @param stormConf Storm configuration
*/
@Override
- public void prepare(Map<String, Object> storm_conf) {
- Map<?, ?> params = (Map<?, ?>) storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PARAMS);
+ public void prepare(Map<String, Object> stormConf) {
+ Map<?, ?> params = (Map<?, ?>) stormConf.get(Config.STORM_GROUP_MAPPING_SERVICE_PARAMS);
Map<String, Set<String>> mapping = (Map<String, Set<String>>) params.get(STORM_FIXED_GROUP_MAPPING);
if (mapping != null) {
cachedGroups.putAll(mapping);
@@ -45,7 +45,7 @@
}
/**
- * Returns list of groups for a user
+ * Returns list of groups for a user.
*
* @param user get groups for this user
* @return list of groups for a given user
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java
index da82d3f..6662827 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/IAuthorizer.java
@@ -17,13 +17,13 @@
/**
* Nimbus could be configured with an authorization plugin. If not specified, all requests are authorized.
*
- * You could specify the authorization plugin via storm parameter. For example:
+ * <p>You could specify the authorization plugin via storm parameter. For example:
*
- * ``` storm -c nimbus.authorization.class=org.apache.storm.security.auth.NoopAuthorizer ... ```
+ * <p>``` storm -c nimbus.authorization.class=org.apache.storm.security.auth.NoopAuthorizer ... ```
*
- * You could also specify it via storm.yaml:
+ * <p>You could also specify it via storm.yaml:
*
- * ```yaml nimbus.authorization.class: org.apache.storm.security.auth.NoopAuthorizer ```
+ * <p>```yaml nimbus.authorization.class: org.apache.storm.security.auth.NoopAuthorizer ```
*/
public interface IAuthorizer {
/**
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/IGroupMappingServiceProvider.java b/storm-client/src/jvm/org/apache/storm/security/auth/IGroupMappingServiceProvider.java
index 05fed9b..3c7edda 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/IGroupMappingServiceProvider.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/IGroupMappingServiceProvider.java
@@ -19,19 +19,17 @@
public interface IGroupMappingServiceProvider {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param topoConf Storm configuration
*/
void prepare(Map<String, Object> topoConf);
/**
- * Get all various group memberships of a given user. Returns EMPTY list in case of non-existing user
+ * Get all various group memberships of a given user. Returns EMPTY list in case of non-existing user.
*
* @param user User's name
* @return group memberships of user
- *
- * @throws IOException
*/
public Set<String> getGroups(String user) throws IOException;
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/IPrincipalToLocal.java b/storm-client/src/jvm/org/apache/storm/security/auth/IPrincipalToLocal.java
index a2d57ae..3163cdb 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/IPrincipalToLocal.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/IPrincipalToLocal.java
@@ -20,7 +20,7 @@
*/
public interface IPrincipalToLocal {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param topoConf Storm configuration
*/
@@ -32,9 +32,9 @@
* @param principal the principal to convert
* @return The local user name.
*/
- default public String toLocal(Principal principal) {
+ default String toLocal(Principal principal) {
return principal == null ? null : toLocal(principal.getName());
}
- public String toLocal(String principalName);
+ String toLocal(String principalName);
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ITransportPlugin.java b/storm-client/src/jvm/org/apache/storm/security/auth/ITransportPlugin.java
index 0e9ed4e..b1118cb 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ITransportPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ITransportPlugin.java
@@ -21,11 +21,11 @@
import org.apache.storm.thrift.transport.TTransportException;
/**
- * Interface for Thrift Transport plugin
+ * Interface for Thrift Transport plugin.
*/
public interface ITransportPlugin {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param type the type of connection this will process.
* @param topoConf Storm configuration
@@ -34,7 +34,7 @@
void prepare(ThriftConnectionType type, Map<String, Object> topoConf, Configuration loginConf);
/**
- * Create a server associated with a given port, service handler, and purpose
+ * Create a server associated with a given port, service handler, and purpose.
*
* @param processor service handler
* @return server
@@ -42,7 +42,7 @@
public TServer getServer(TProcessor processor) throws IOException, TTransportException;
/**
- * Connect to the specified server via framed transport
+ * Connect to the specified server via framed transport.
*
* @param transport The underlying Thrift transport.
* @param serverHost server host
@@ -52,7 +52,9 @@
public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException;
/**
- * @return The port this transport is using. This is not known until {@link #getServer(org.apache.storm.thrift.TProcessor)} has been called.
+ * Get port.
+ * @return The port this transport is using. This is not known until
+ * {@link #getServer(org.apache.storm.thrift.TProcessor)} has been called
*/
public int getPort();
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/KerberosPrincipalToLocal.java b/storm-client/src/jvm/org/apache/storm/security/auth/KerberosPrincipalToLocal.java
index ea341f0..5ca3865 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/KerberosPrincipalToLocal.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/KerberosPrincipalToLocal.java
@@ -15,12 +15,12 @@
import java.util.Map;
/**
- * Map a kerberos principal to a local user
+ * Map a kerberos principal to a local user.
*/
public class KerberosPrincipalToLocal implements IPrincipalToLocal {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param topoConf Storm configuration
*/
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ReqContext.java b/storm-client/src/jvm/org/apache/storm/security/auth/ReqContext.java
index 0be4810..11a1cdf 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ReqContext.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ReqContext.java
@@ -28,9 +28,12 @@
import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
/**
- * context request context includes info about:
- *
- * 1. remote address, 2. remote subject and primary principal 3. request ID
+ * Request context. Context request context includes info about:
+ * <ol>
+ * <li>remote address</li>
+ * <li>remote subject and primary principal
+ * <li>request ID</li>
+ * </ol>
*/
public class ReqContext {
private static final AtomicInteger uniqueId = new AtomicInteger(0);
@@ -39,21 +42,21 @@
ThreadLocal.withInitial(() -> new ReqContext(AccessController.getContext()));
private Subject subject;
private InetAddress remoteAddr;
- private final int reqID;
+ private final int reqId;
private Principal realPrincipal;
//private constructor
@VisibleForTesting
- public ReqContext(AccessControlContext acl_ctxt) {
- subject = Subject.getSubject(acl_ctxt);
- reqID = uniqueId.incrementAndGet();
+ public ReqContext(AccessControlContext aclCtxt) {
+ subject = Subject.getSubject(aclCtxt);
+ reqId = uniqueId.incrementAndGet();
}
//private constructor
@VisibleForTesting
public ReqContext(Subject sub) {
subject = sub;
- reqID = uniqueId.incrementAndGet();
+ reqId = uniqueId.incrementAndGet();
}
/**
@@ -63,11 +66,12 @@
public ReqContext(ReqContext other) {
subject = other.subject;
remoteAddr = other.remoteAddr;
- reqID = other.reqID;
+ reqId = other.reqId;
realPrincipal = other.realPrincipal;
}
/**
+ * Get context.
* @return a request context associated with current thread
*/
public static ReqContext context() {
@@ -83,17 +87,17 @@
@Override
public String toString() {
- return "ReqContext{" +
- "realPrincipal=" + ((realPrincipal != null) ? realPrincipal.getName() : "null") +
- ", reqID=" + reqID +
- ", remoteAddr=" + remoteAddr +
- ", authZPrincipal=" + ((principal() != null) ? principal().getName() : "null") +
- ", ThreadId=" + Thread.currentThread().toString() +
- '}';
+ return "ReqContext{"
+ + "realPrincipal=" + ((realPrincipal != null) ? realPrincipal.getName() : "null")
+ + ", reqId=" + reqId
+ + ", remoteAddr=" + remoteAddr
+ + ", authZPrincipal=" + ((principal() != null) ? principal().getName() : "null")
+ + ", ThreadId=" + Thread.currentThread().toString()
+ + '}';
}
/**
- * client address
+ * client address.
*/
public void setRemoteAddress(InetAddress addr) {
remoteAddr = addr;
@@ -104,21 +108,21 @@
}
/**
- * Set remote subject explicitly
+ * Set remote subject explicitly.
*/
public void setSubject(Subject subject) {
this.subject = subject;
}
/**
- * Retrieve client subject associated with this request context
+ * Retrieve client subject associated with this request context.
*/
public Subject subject() {
return subject;
}
/**
- * The primary principal associated current subject
+ * The primary principal associated current subject.
*/
public Principal principal() {
if (subject == null) {
@@ -143,6 +147,7 @@
}
/**
+ * Check whether context is impersonating.
* @return true if this request is an impersonation request.
*/
public boolean isImpersonating() {
@@ -150,9 +155,10 @@
}
/**
- * request ID of this request
+ * request ID of this request.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public int requestID() {
- return reqID;
+ return reqId;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ShellBasedGroupsMapping.java b/storm-client/src/jvm/org/apache/storm/security/auth/ShellBasedGroupsMapping.java
index 1991a8e..31d00dc 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ShellBasedGroupsMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ShellBasedGroupsMapping.java
@@ -49,7 +49,7 @@
}
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param topoConf Storm configuration
*/
@@ -61,7 +61,7 @@
}
/**
- * Returns list of groups for a user
+ * Returns list of groups for a user.
*
* @param user get groups for this user
* @return list of groups for a given user
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java b/storm-client/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java
index 46617e0..21a6ce1 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/SimpleTransportPlugin.java
@@ -42,20 +42,20 @@
/**
* Simple transport for Thrift plugin.
*
- * This plugin is designed to be backward compatible with existing Storm code.
+ * <p>This plugin is designed to be backward compatible with existing Storm code.
*/
public class SimpleTransportPlugin implements ITransportPlugin {
private static final Logger LOG = LoggerFactory.getLogger(SimpleTransportPlugin.class);
protected ThriftConnectionType type;
protected Map<String, Object> topoConf;
- protected Configuration login_conf;
+ protected Configuration loginConf;
private int port;
@Override
public void prepare(ThriftConnectionType type, Map<String, Object> topoConf, Configuration loginConf) {
this.type = type;
this.topoConf = topoConf;
- this.login_conf = loginConf;
+ this.loginConf = loginConf;
}
@Override
@@ -67,30 +67,31 @@
int maxBufferSize = type.getMaxBufferSize(topoConf);
Integer queueSize = type.getQueueSize(topoConf);
- THsHaServer.Args server_args = new THsHaServer.Args(serverTransport).
- processor(new SimpleWrapProcessor(processor)).
- maxWorkerThreads(numWorkerThreads).
- protocolFactory(
- new TBinaryProtocol.Factory(false, true, maxBufferSize,
- -1));
+ THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport)
+ .processor(new SimpleWrapProcessor(processor))
+ .maxWorkerThreads(numWorkerThreads)
+ .protocolFactory(new TBinaryProtocol.Factory(false,
+ true,
+ maxBufferSize,
+ -1));
- server_args.maxReadBufferBytes = maxBufferSize;
+ serverArgs.maxReadBufferBytes = maxBufferSize;
if (queueSize != null) {
- server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads,
+ serverArgs.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads,
60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize)));
}
//construct THsHaServer
- return new THsHaServer(server_args);
+ return new THsHaServer(serverArgs);
}
/**
- * Connect to the specified server via framed transport
+ * Connect to the specified server via framed transport.
*
- * @param transport The underlying Thrift transport.
- * @param serverHost unused.
- * @param asUser unused.
+ * @param transport The underlying Thrift transport
+ * @param serverHost unused
+ * @param asUser unused
*/
@Override
public TTransport connect(TTransport transport, String serverHost, String asUser) throws TTransportException {
@@ -106,6 +107,7 @@
}
/**
+ * Get default subject.
* @return the subject that will be used for all connections
*/
protected Subject getDefaultSubject() {
@@ -118,7 +120,7 @@
}
/**
- * Processor that populate simple transport info into ReqContext, and then invoke a service handler
+ * Processor that populate simple transport info into ReqContext, and then invoke a service handler.
*/
private class SimpleWrapProcessor implements TProcessor {
final TProcessor wrapped;
@@ -130,12 +132,12 @@
@Override
public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException {
//populating request context
- ReqContext req_context = ReqContext.context();
+ ReqContext reqContext = ReqContext.context();
TTransport trans = inProt.getTransport();
if (trans instanceof TMemoryInputTransport) {
try {
- req_context.setRemoteAddress(InetAddress.getLocalHost());
+ reqContext.setRemoteAddress(InetAddress.getLocalHost());
} catch (UnknownHostException e) {
throw new RuntimeException(e);
}
@@ -143,7 +145,7 @@
TSocket tsocket = (TSocket) trans;
//remote address
Socket socket = tsocket.getSocket();
- req_context.setRemoteAddress(socket.getInetAddress());
+ reqContext.setRemoteAddress(socket.getInetAddress());
}
//anonymous user
@@ -166,7 +168,7 @@
s = new Subject(true, principals, new HashSet<>(), new HashSet<>());
}
}
- req_context.setSubject(s);
+ reqContext.setSubject(s);
//invoke service handler
return wrapped.process(inProt, outProt);
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/SingleUserPrincipal.java b/storm-client/src/jvm/org/apache/storm/security/auth/SingleUserPrincipal.java
index 2e117ba..2885103 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/SingleUserPrincipal.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/SingleUserPrincipal.java
@@ -19,20 +19,20 @@
*/
public class SingleUserPrincipal implements Principal {
- private final String _userName;
+ private final String userName;
public SingleUserPrincipal(String userName) {
- _userName = userName;
+ this.userName = userName;
}
@Override
public boolean equals(Object another) {
- return another instanceof SingleUserPrincipal && _userName.equals(((SingleUserPrincipal) another)._userName);
+ return another instanceof SingleUserPrincipal && userName.equals(((SingleUserPrincipal) another).userName);
}
@Override
public String getName() {
- return _userName;
+ return userName;
}
@Override
@@ -42,6 +42,6 @@
@Override
public int hashCode() {
- return _userName.hashCode();
+ return userName.hashCode();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java b/storm-client/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java
index ad5e5a9..5b09aac 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/TBackoffConnect.java
@@ -21,15 +21,15 @@
public class TBackoffConnect {
private static final Logger LOG = LoggerFactory.getLogger(TBackoffConnect.class);
- private int _completedRetries = 0;
- private int _retryTimes;
+ private int completedRetries = 0;
+ private int retryTimes;
private StormBoundedExponentialBackoffRetry waitGrabber;
- private boolean _retryForever = false;
+ private boolean retryForever = false;
public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling, boolean retryForever) {
- _retryForever = retryForever;
- _retryTimes = retryTimes;
+ this.retryForever = retryForever;
+ this.retryTimes = retryTimes;
waitGrabber = new StormBoundedExponentialBackoffRetry(retryInterval,
retryIntervalCeiling,
retryTimes);
@@ -59,19 +59,19 @@
throw new RuntimeException(ex);
}
try {
- long sleeptime = waitGrabber.getSleepTimeMs(_completedRetries, 0);
+ long sleeptime = waitGrabber.getSleepTimeMs(completedRetries, 0);
- LOG.debug("Failed to connect. Retrying... (" + Integer.toString(_completedRetries) + ") in " + Long.toString(sleeptime) + "ms");
+ LOG.debug("Failed to connect. Retrying... (" + Integer.toString(completedRetries) + ") in " + Long.toString(sleeptime) + "ms");
Thread.sleep(sleeptime);
} catch (InterruptedException e) {
LOG.info("Nimbus connection retry interrupted.");
}
- _completedRetries++;
+ completedRetries++;
}
private boolean canRetry() {
- return _retryForever || (_completedRetries < _retryTimes);
+ return retryForever || (completedRetries < retryTimes);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
index 9ad5ba3..a9d9129 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
@@ -15,22 +15,22 @@
import java.util.Map;
import javax.security.auth.login.Configuration;
import org.apache.storm.Config;
-import org.apache.storm.utils.ObjectReader;
import org.apache.storm.thrift.protocol.TBinaryProtocol;
import org.apache.storm.thrift.protocol.TProtocol;
import org.apache.storm.thrift.transport.TSocket;
import org.apache.storm.thrift.transport.TTransport;
+import org.apache.storm.utils.ObjectReader;
public class ThriftClient implements AutoCloseable {
- protected TProtocol _protocol;
- protected boolean _retryForever = false;
- private TTransport _transport;
- private String _host;
- private Integer _port;
- private Integer _timeout;
- private Map _conf;
- private ThriftConnectionType _type;
- private String _asUser;
+ protected TProtocol protocol;
+ protected boolean retryForever = false;
+ private TTransport transport;
+ private String host;
+ private Integer port;
+ private Integer timeout;
+ private Map conf;
+ private ThriftConnectionType type;
+ private String asUser;
public ThriftClient(Map<String, Object> topoConf, ThriftConnectionType type, String host) {
this(topoConf, type, host, null, null, null);
@@ -59,68 +59,69 @@
throw new IllegalArgumentException("invalid port: " + port);
}
- _host = host;
- _port = port;
- _timeout = timeout;
- _conf = topoConf;
- _type = type;
- _asUser = asUser;
+ this.host = host;
+ this.port = port;
+ this.timeout = timeout;
+ conf = topoConf;
+ this.type = type;
+ this.asUser = asUser;
if (!type.isFake()) {
reconnect();
}
}
public synchronized TTransport transport() {
- return _transport;
+ return transport;
}
public synchronized void reconnect() {
close();
TSocket socket = null;
try {
- socket = new TSocket(_host, _port);
- if (_timeout != null) {
- socket.setTimeout(_timeout);
+ socket = new TSocket(host, port);
+ if (timeout != null) {
+ socket.setTimeout(timeout);
}
//locate login configuration
- Configuration login_conf = ClientAuthUtils.getConfiguration(_conf);
+ Configuration loginConf = ClientAuthUtils.getConfiguration(conf);
//construct a transport plugin
- ITransportPlugin transportPlugin = ClientAuthUtils.getTransportPlugin(_type, _conf, login_conf);
+ ITransportPlugin transportPlugin = ClientAuthUtils.getTransportPlugin(type, conf, loginConf);
//TODO get this from type instead of hardcoding to Nimbus.
//establish client-server transport via plugin
//do retries if the connect fails
TBackoffConnect connectionRetry
= new TBackoffConnect(
- ObjectReader.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_TIMES)),
- ObjectReader.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)),
- ObjectReader.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING)),
- _retryForever);
- _transport = connectionRetry.doConnectWithRetry(transportPlugin, socket, _host, _asUser);
+ ObjectReader.getInt(conf.get(Config.STORM_NIMBUS_RETRY_TIMES)),
+ ObjectReader.getInt(conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)),
+ ObjectReader.getInt(conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING)),
+ retryForever);
+ transport = connectionRetry.doConnectWithRetry(transportPlugin, socket, host, asUser);
} catch (Exception ex) {
// close the socket, which releases connection if it has created any.
if (socket != null) {
try {
socket.close();
} catch (Exception e) {
+ //ignore
}
}
throw new RuntimeException(ex);
}
- _protocol = null;
- if (_transport != null) {
- _protocol = new TBinaryProtocol(_transport);
+ protocol = null;
+ if (transport != null) {
+ protocol = new TBinaryProtocol(transport);
}
}
@Override
public synchronized void close() {
- if (_transport != null) {
- _transport.close();
- _transport = null;
- _protocol = null;
+ if (transport != null) {
+ transport.close();
+ transport = null;
+ protocol = null;
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
index f9dc775..b6d7d13 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
@@ -36,7 +36,7 @@
private final String transConf;
private final String portConf;
- private final String qConf;
+ private final String queueConf;
private final String threadsConf;
private final String buffConf;
private final String socketTimeoutConf;
@@ -48,18 +48,18 @@
this(null, null, null, null, null, null, true, null, false);
}
- ThriftConnectionType(String transConf, String portConf, String qConf,
+ ThriftConnectionType(String transConf, String portConf, String queueConf,
String threadsConf, String buffConf, String socketTimeoutConf,
WorkerTokenServiceType wtType, boolean impersonationAllowed) {
- this(transConf, portConf, qConf, threadsConf, buffConf, socketTimeoutConf, false, wtType, impersonationAllowed);
+ this(transConf, portConf, queueConf, threadsConf, buffConf, socketTimeoutConf, false, wtType, impersonationAllowed);
}
- ThriftConnectionType(String transConf, String portConf, String qConf,
+ ThriftConnectionType(String transConf, String portConf, String queueConf,
String threadsConf, String buffConf, String socketTimeoutConf, boolean isFake,
WorkerTokenServiceType wtType, boolean impersonationAllowed) {
this.transConf = transConf;
this.portConf = portConf;
- this.qConf = qConf;
+ this.queueConf = queueConf;
this.threadsConf = threadsConf;
this.buffConf = buffConf;
this.socketTimeoutConf = socketTimeoutConf;
@@ -88,10 +88,10 @@
}
public Integer getQueueSize(Map<String, Object> conf) {
- if (qConf == null) {
+ if (queueConf == null) {
return null;
}
- return (Integer) conf.get(qConf);
+ return (Integer) conf.get(queueConf);
}
public int getNumThreads(Map<String, Object> conf) {
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftServer.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftServer.java
index e83d37d..f6102bc 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftServer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftServer.java
@@ -66,6 +66,7 @@
}
/**
+ * Check whether serving.
* @return true if ThriftServer is listening to requests?
*/
public boolean isServing() {
@@ -90,6 +91,7 @@
}
/**
+ * Get port.
* @return The port this server is/will be listening on
*/
public int getPort() {
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCAuthorizerBase.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCAuthorizerBase.java
index 9019e45..dcb1d48 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCAuthorizerBase.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCAuthorizerBase.java
@@ -18,6 +18,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public abstract class DRPCAuthorizerBase implements IAuthorizer {
public static final Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class);
@@ -29,9 +30,9 @@
@Override
public abstract void prepare(Map<String, Object> conf);
- abstract protected boolean permitClientRequest(ReqContext context, String operation, Map<String, Object> params);
+ protected abstract boolean permitClientRequest(ReqContext context, String operation, Map<String, Object> params);
- abstract protected boolean permitInvocationRequest(ReqContext context, String operation, Map<String, Object> params);
+ protected abstract boolean permitInvocationRequest(ReqContext context, String operation, Map<String, Object> params);
/**
* Authorizes request from to the DRPC server.
@@ -44,14 +45,13 @@
public boolean permit(ReqContext context, String operation, Map<String, Object> params) {
if ("execute".equals(operation)) {
return permitClientRequest(context, operation, params);
- } else if ("failRequest".equals(operation) ||
- "fetchRequest".equals(operation) ||
- "result".equals(operation)) {
+ } else if ("failRequest".equals(operation)
+ || "fetchRequest".equals(operation)
+ || "result".equals(operation)) {
return permitInvocationRequest(context, operation, params);
}
// Deny unsupported operations.
- LOG.warn("Denying unsupported operation \"" + operation + "\" from " +
- context.remoteAddress());
+ LOG.warn("Denying unsupported operation \"" + operation + "\" from " + context.remoteAddress());
return false;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
index 3912fec..00d50c4 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
@@ -28,6 +28,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class DRPCSimpleACLAuthorizer extends DRPCAuthorizerBase {
public static final Logger LOG =
LoggerFactory.getLogger(DRPCSimpleACLAuthorizer.class);
@@ -36,20 +37,20 @@
public static final String INVOCATION_USER_KEY = "invocation.user";
public static final String FUNCTION_KEY = "function.name";
- protected String _aclFileName = "";
- protected IPrincipalToLocal _ptol;
- protected boolean _permitWhenMissingFunctionEntry = false;
- private volatile Map<String, AclFunctionEntry> _acl = null;
- private volatile long _lastUpdate = 0;
+ protected String aclFileName = "";
+ protected IPrincipalToLocal ptol;
+ protected boolean permitWhenMissingFunctionEntry = false;
+ private volatile Map<String, AclFunctionEntry> acl = null;
+ private volatile long lastUpdate = 0;
protected Map<String, AclFunctionEntry> readAclFromConfig() {
- //Thread safety is mostly around _acl. If _acl needs to be updated it is changed atomically
+ //Thread safety is mostly around acl. If acl needs to be updated it is changed atomically
//More then one thread may be trying to update it at a time, but that is OK, because the
//change is atomic
long now = System.currentTimeMillis();
- if ((now - 5000) > _lastUpdate || _acl == null) {
+ if ((now - 5000) > lastUpdate || acl == null) {
Map<String, AclFunctionEntry> acl = new HashMap<>();
- Map<String, Object> conf = Utils.findAndReadConfigFile(_aclFileName);
+ Map<String, Object> conf = Utils.findAndReadConfigFile(aclFileName);
if (conf.containsKey(Config.DRPC_AUTHORIZER_ACL)) {
Map<String, Map<String, ?>> confAcl =
(Map<String, Map<String, ?>>)
@@ -57,33 +58,32 @@
for (Map.Entry<String, Map<String, ?>> entry : confAcl.entrySet()) {
Map<String, ?> val = entry.getValue();
- Collection<String> clientUsers =
- val.containsKey(CLIENT_USERS_KEY) ?
- (Collection<String>) val.get(CLIENT_USERS_KEY) : null;
- String invocationUser =
- val.containsKey(INVOCATION_USER_KEY) ?
- (String) val.get(INVOCATION_USER_KEY) : null;
+ Collection<String> clientUsers = val.containsKey(CLIENT_USERS_KEY)
+ ? (Collection<String>) val.get(CLIENT_USERS_KEY)
+ : null;
+ String invocationUser = val.containsKey(INVOCATION_USER_KEY)
+ ? (String) val.get(INVOCATION_USER_KEY)
+ : null;
acl.put(entry.getKey(),
new AclFunctionEntry(clientUsers, invocationUser));
}
- } else if (!_permitWhenMissingFunctionEntry) {
- LOG.warn("Requiring explicit ACL entries, but none given. " +
- "Therefore, all operations will be denied.");
+ } else if (!permitWhenMissingFunctionEntry) {
+ LOG.warn("Requiring explicit ACL entries, but none given. Therefore, all operations will be denied.");
}
- _acl = acl;
- _lastUpdate = System.currentTimeMillis();
+ this.acl = acl;
+ lastUpdate = System.currentTimeMillis();
}
- return _acl;
+ return acl;
}
@Override
public void prepare(Map<String, Object> conf) {
Boolean isStrict =
(Boolean) conf.get(Config.DRPC_AUTHORIZER_ACL_STRICT);
- _permitWhenMissingFunctionEntry =
+ permitWhenMissingFunctionEntry =
(isStrict != null && !isStrict);
- _aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME);
- _ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
+ aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME);
+ ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
}
private String getUserFromContext(ReqContext context) {
@@ -98,7 +98,7 @@
private String getLocalUserFromContext(ReqContext context) {
if (context != null) {
- return _ptol.toLocal(context.principal());
+ return ptol.toLocal(context.principal());
}
return null;
}
@@ -109,7 +109,7 @@
String function = (String) params.get(FUNCTION_KEY);
if (function != null && !function.isEmpty()) {
AclFunctionEntry entry = acl.get(function);
- if (entry == null && _permitWhenMissingFunctionEntry) {
+ if (entry == null && permitWhenMissingFunctionEntry) {
return true;
}
if (entry != null) {
@@ -124,16 +124,13 @@
String principal = getUserFromContext(context);
String user = getLocalUserFromContext(context);
if (value == null) {
- LOG.warn("Configuration for function '" + function + "' is " +
- "invalid: it should have both an invocation user " +
- "and a list of client users defined.");
- } else if (value instanceof Set &&
- (((Set<String>) value).contains(principal) ||
- ((Set<String>) value).contains(user))) {
+ LOG.warn("Configuration for function '" + function + "' is "
+ + "invalid: it should have both an invocation user and a list of client users defined.");
+ } else if (value instanceof Set && (((Set<String>) value).contains(principal)
+ || ((Set<String>) value).contains(user))) {
return true;
- } else if (value instanceof String &&
- (value.equals(principal) ||
- value.equals(user))) {
+ } else if (value instanceof String && (value.equals(principal)
+ || value.equals(user))) {
return true;
}
}
@@ -155,13 +152,14 @@
@VisibleForTesting
public static class AclFunctionEntry {
- final public Set<String> clientUsers;
- final public String invocationUser;
+ public final Set<String> clientUsers;
+ public final String invocationUser;
public AclFunctionEntry(Collection<String> clientUsers,
String invocationUser) {
- this.clientUsers = (clientUsers != null) ?
- new HashSet<>(clientUsers) : new HashSet<String>();
+ this.clientUsers = (clientUsers != null)
+ ? new HashSet<>(clientUsers)
+ : new HashSet<String>();
this.invocationUser = invocationUser;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DenyAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DenyAuthorizer.java
index e51a7b0..8ce426a 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DenyAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/DenyAuthorizer.java
@@ -17,12 +17,12 @@
import org.apache.storm.security.auth.ReqContext;
/**
- * An authorization implementation that denies everything, for testing purposes
+ * An authorization implementation that denies everything, for testing purposes.
*/
public class DenyAuthorizer implements IAuthorizer {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param conf Storm configuration
*/
@@ -31,7 +31,7 @@
}
/**
- * permit() method is invoked for each incoming Thrift request
+ * permit() method is invoked for each incoming Thrift request.
*
* @param context request context
* @param operation operation name
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/ImpersonationAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/ImpersonationAuthorizer.java
index aa52311..b07e9bd 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/ImpersonationAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/ImpersonationAuthorizer.java
@@ -33,9 +33,10 @@
public class ImpersonationAuthorizer implements IAuthorizer {
protected static final String WILD_CARD = "*";
private static final Logger LOG = LoggerFactory.getLogger(ImpersonationAuthorizer.class);
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
protected Map<String, ImpersonationACL> userImpersonationACL;
- protected IPrincipalToLocal _ptol;
- protected IGroupMappingServiceProvider _groupMappingProvider;
+ protected IPrincipalToLocal ptol;
+ protected IGroupMappingServiceProvider groupMappingProvider;
@Override
public void prepare(Map<String, Object> conf) {
@@ -53,8 +54,8 @@
}
}
- _ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
- _groupMappingProvider = ClientAuthUtils.getGroupMappingServiceProviderPlugin(conf);
+ ptol = ClientAuthUtils.getPrincipalToLocalPlugin(conf);
+ groupMappingProvider = ClientAuthUtils.getGroupMappingServiceProviderPlugin(conf);
}
@Override
@@ -65,8 +66,8 @@
}
String impersonatingPrincipal = context.realPrincipal().getName();
- String impersonatingUser = _ptol.toLocal(context.realPrincipal());
- String userBeingImpersonated = _ptol.toLocal(context.principal());
+ String impersonatingUser = ptol.toLocal(context.realPrincipal());
+ String userBeingImpersonated = ptol.toLocal(context.principal());
InetAddress remoteAddress = context.remoteAddress();
LOG.info("user = {}, principal = {} is attempting to impersonate user = {} for operation = {} from host = {}",
@@ -76,15 +77,19 @@
* no config is present for impersonating principal or user, do not permit impersonation.
*/
if (!userImpersonationACL.containsKey(impersonatingPrincipal) && !userImpersonationACL.containsKey(impersonatingUser)) {
- LOG.info(
- "user = {}, principal = {} is trying to impersonate user {}, but config {} does not have entry for impersonating user or " +
- "principal." +
- "Please see SECURITY.MD to learn how to configure users for impersonation."
- , impersonatingUser, impersonatingPrincipal, userBeingImpersonated, Config.NIMBUS_IMPERSONATION_ACL);
+ LOG.info("user = {}, principal = {} is trying to impersonate user {}, but config {} does not have entry "
+ + "for impersonating user or principal."
+ + "Please see SECURITY.MD to learn how to configure users for impersonation.",
+ impersonatingUser,
+ impersonatingPrincipal,
+ userBeingImpersonated,
+ Config.NIMBUS_IMPERSONATION_ACL);
return false;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
ImpersonationACL principalACL = userImpersonationACL.get(impersonatingPrincipal);
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
ImpersonationACL userACL = userImpersonationACL.get(impersonatingUser);
Set<String> authorizedHosts = new HashSet<>();
@@ -120,10 +125,10 @@
}
private boolean isAllowedToImpersonateFromHost(Set<String> authorizedHosts, InetAddress remoteAddress) {
- return authorizedHosts.contains(WILD_CARD) ||
- authorizedHosts.contains(remoteAddress.getCanonicalHostName()) ||
- authorizedHosts.contains(remoteAddress.getHostName()) ||
- authorizedHosts.contains(remoteAddress.getHostAddress());
+ return authorizedHosts.contains(WILD_CARD)
+ || authorizedHosts.contains(remoteAddress.getCanonicalHostName())
+ || authorizedHosts.contains(remoteAddress.getHostName())
+ || authorizedHosts.contains(remoteAddress.getHostAddress());
}
private boolean isAllowedToImpersonateUser(Set<String> authorizedGroups, String userBeingImpersonated) {
@@ -133,7 +138,7 @@
Set<String> groups;
try {
- groups = _groupMappingProvider.getGroups(userBeingImpersonated);
+ groups = groupMappingProvider.getGroups(userBeingImpersonated);
} catch (IOException e) {
throw new RuntimeException("failed to get groups for user " + userBeingImpersonated);
}
@@ -151,6 +156,7 @@
return false;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
protected static class ImpersonationACL {
public String impersonatingUser;
//Groups this user is authorized to impersonate.
@@ -166,11 +172,11 @@
@Override
public String toString() {
- return "ImpersonationACL{" +
- "impersonatingUser='" + impersonatingUser + '\'' +
- ", authorizedGroups=" + authorizedGroups +
- ", authorizedHosts=" + authorizedHosts +
- '}';
+ return "ImpersonationACL{"
+ + "impersonatingUser='" + impersonatingUser + '\''
+ + ", authorizedGroups=" + authorizedGroups
+ + ", authorizedHosts=" + authorizedHosts
+ + '}';
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/NoopAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/NoopAuthorizer.java
index c984f3a..bef7a76 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/NoopAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/NoopAuthorizer.java
@@ -22,7 +22,7 @@
public class NoopAuthorizer implements IAuthorizer {
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param conf Storm configuration
*/
@@ -31,7 +31,7 @@
}
/**
- * permit() method is invoked for each incoming Thrift request
+ * permit() method is invoked for each incoming Thrift request.
*
* @param context request context includes info about
* @param operation operation name
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
index df94fe7..f639b28 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleACLAuthorizer.java
@@ -30,6 +30,7 @@
/**
* An authorization implementation that simply checks if a user is allowed to perform specific operations.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class SimpleACLAuthorizer implements IAuthorizer {
private static final Logger LOG = LoggerFactory.getLogger(SimpleACLAuthorizer.class);
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
index 4688ed9..9816056 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
@@ -27,7 +27,7 @@
protected Set<String> users;
/**
- * Invoked once immediately after construction
+ * Invoked once immediately after construction.
*
* @param conf Storm configuration
*/
@@ -40,7 +40,7 @@
}
/**
- * `permit()` method is invoked for each incoming Thrift request
+ * `permit()` method is invoked for each incoming Thrift request.
*
* @param context request context includes info about
* @param operation operation name
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java
index d8e8805..5875ff0 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/authorizer/SupervisorSimpleACLAuthorizer.java
@@ -30,6 +30,7 @@
/**
* An authorization implementation that simply checks if a user is allowed to perform specific operations.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class SupervisorSimpleACLAuthorizer implements IAuthorizer {
private static final Logger LOG = LoggerFactory.getLogger(SupervisorSimpleACLAuthorizer.class);
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index 371f271..e93b1c4 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -37,6 +37,7 @@
/**
* Automatically take a user's TGT, and push it, and renew it in Nimbus.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetricsRegistrant {
protected static final AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<>();
private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
@@ -44,6 +45,7 @@
private Map<String, Object> conf;
private Map<String, String> credentials;
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static KerberosTicket getTGT(Subject subject) {
Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
for (KerberosTicket ticket : tickets) {
@@ -55,6 +57,16 @@
return null;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
+ public static KerberosTicket getTGT(Map<String, String> credentials) {
+ KerberosTicket ret = null;
+ if (credentials != null && credentials.containsKey("TGT") && credentials.get("TGT") != null) {
+ ret = ClientAuthUtils.deserializeKerberosTicket(DatatypeConverter.parseBase64Binary(credentials.get("TGT")));
+ }
+ return ret;
+ }
+
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static void saveTGT(KerberosTicket tgt, Map<String, String> credentials) {
try {
@@ -65,14 +77,6 @@
}
}
- public static KerberosTicket getTGT(Map<String, String> credentials) {
- KerberosTicket ret = null;
- if (credentials != null && credentials.containsKey("TGT") && credentials.get("TGT") != null) {
- ret = ClientAuthUtils.deserializeKerberosTicket(DatatypeConverter.parseBase64Binary(credentials.get("TGT")));
- }
- return ret;
- }
-
public static void clearCredentials(Subject subject, KerberosTicket tgt) {
Set<Object> creds = subject.getPrivateCredentials();
synchronized (creds) {
@@ -116,12 +120,12 @@
this.credentials = credentials;
//Log the user in and get the TGT
try {
- Configuration login_conf = ClientAuthUtils.getConfiguration(conf);
- ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf);
+ Configuration loginConf = ClientAuthUtils.getConfiguration(conf);
+ ClientCallbackHandler clientCallbackHandler = new ClientCallbackHandler(loginConf);
//login our user
- Configuration.setConfiguration(login_conf);
- LoginContext lc = new LoginContext(ClientAuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
+ Configuration.setConfiguration(loginConf);
+ LoginContext lc = new LoginContext(ClientAuthUtils.LOGIN_CONTEXT_CLIENT, clientCallbackHandler);
try {
lc.login();
final Subject subject = lc.getSubject();
@@ -129,7 +133,7 @@
if (tgt == null) { //error
throw new RuntimeException("Fail to verify user principal with section \""
- + ClientAuthUtils.LOGIN_CONTEXT_CLIENT + "\" in login configuration file " + login_conf);
+ + ClientAuthUtils.LOGIN_CONTEXT_CLIENT + "\" in login configuration file " + loginConf);
}
if (!tgt.isForwardable()) {
@@ -167,6 +171,7 @@
loginHadoopUser(subject);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private void populateSubjectWithTGT(Subject subject, Map<String, String> credentials) {
KerberosTicket tgt = getTGT(credentials);
if (tgt != null) {
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
index e7b6eac..cc070f5 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
@@ -25,8 +25,9 @@
/**
- * Custom LoginModule to enable Auto Login based on cached ticket
+ * Custom LoginModule to enable Auto Login based on cached ticket.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AutoTGTKrb5LoginModule implements LoginModule {
private static final Logger LOG = LoggerFactory.getLogger(AutoTGTKrb5LoginModule.class);
protected KerberosTicket kerbTicket = null;
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
index 76b21c8..16b8eb0 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
@@ -18,6 +18,7 @@
/**
* Custom LoginModule extended for testing.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AutoTGTKrb5LoginModuleTest extends AutoTGTKrb5LoginModule {
public Principal client = null;
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ClientCallbackHandler.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ClientCallbackHandler.java
index bf06d3c..64673ce 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ClientCallbackHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ClientCallbackHandler.java
@@ -32,18 +32,15 @@
private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class);
/**
- * Constructor based on a JAAS configuration
+ * Constructor based on a JAAS configuration.
*
- * For digest, you should have a pair of user name and password defined in this figgure.
- *
- * @param configuration
- * @throws IOException
+ * <p>For digest, you should have a pair of user name and password defined in this figgure.
*/
public ClientCallbackHandler(Configuration configuration) throws IOException {
if (configuration == null) {
return;
}
- AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(ClientAuthUtils.LOGIN_CONTEXT_CLIENT);
+ AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(ClientAuthUtils.LOGIN_CONTEXT_CLIENT);
if (configurationEntries == null) {
String errorMessage = "Could not find a '" + ClientAuthUtils.LOGIN_CONTEXT_CLIENT
+ "' entry in this configuration: Client cannot start.";
@@ -53,7 +50,7 @@
}
/**
- * This method is invoked by SASL for authentication challenges
+ * This method is invoked by SASL for authentication challenges.
*
* @param callbacks a collection of challenge callbacks
*/
@@ -64,20 +61,20 @@
LOG.debug("name callback");
} else if (c instanceof PasswordCallback) {
LOG.debug("password callback");
- LOG.warn("Could not login: the client is being asked for a password, but the " +
- " client code does not currently support obtaining a password from the user." +
- " Make sure that the client is configured to use a ticket cache (using" +
- " the JAAS configuration setting 'useTicketCache=true)' and restart the client. If" +
- " you still get this message after that, the TGT in the ticket cache has expired and must" +
- " be manually refreshed. To do so, first determine if you are using a password or a" +
- " keytab. If the former, run kinit in a Unix shell in the environment of the user who" +
- " is running this client using the command" +
- " 'kinit <princ>' (where <princ> is the name of the client's Kerberos principal)." +
- " If the latter, do" +
- " 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the Kerberos principal, and" +
- " <keytab> is the location of the keytab file). After manually refreshing your cache," +
- " restart this client. If you continue to see this message after manually refreshing" +
- " your cache, ensure that your KDC host's clock is in sync with this host's clock.");
+ LOG.warn("Could not login: the client is being asked for a password, but the "
+ + " client code does not currently support obtaining a password from the user."
+ + " Make sure that the client is configured to use a ticket cache (using"
+ + " the JAAS configuration setting 'useTicketCache=true)' and restart the client. If"
+ + " you still get this message after that, the TGT in the ticket cache has expired and must"
+ + " be manually refreshed. To do so, first determine if you are using a password or a"
+ + " keytab. If the former, run kinit in a Unix shell in the environment of the user who"
+ + " is running this client using the command"
+ + " 'kinit <princ>' (where <princ> is the name of the client's Kerberos principal)."
+ + " If the latter, do"
+ + " 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the Kerberos principal, and"
+ + " <keytab> is the location of the keytab file). After manually refreshing your cache,"
+ + " restart this client. If you continue to see this message after manually refreshing"
+ + " your cache, ensure that your KDC host's clock is in sync with this host's clock.");
} else if (c instanceof AuthorizeCallback) {
LOG.debug("authorization callback");
AuthorizeCallback ac = (AuthorizeCallback) c;
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
index 7cf109b..2cd43ba 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
@@ -58,7 +58,7 @@
workerTokenAuthorizer = new WorkerTokenAuthorizer(conf, type);
}
//create an authentication callback handler
- CallbackHandler server_callback_handler = new ServerCallbackHandler(loginConf, impersonationAllowed);
+ CallbackHandler serverCallbackHandler = new ServerCallbackHandler(loginConf, impersonationAllowed);
//login our principal
Subject subject = null;
@@ -66,7 +66,7 @@
//specify a configuration object to be used
Configuration.setConfiguration(loginConf);
//now login
- Login login = new Login(ClientAuthUtils.LOGIN_CONTEXT_SERVER, server_callback_handler);
+ Login login = new Login(ClientAuthUtils.LOGIN_CONTEXT_SERVER, serverCallbackHandler);
subject = login.getSubject();
login.startThreadIfNeeded();
} catch (LoginException ex) {
@@ -91,7 +91,7 @@
//create a transport factory that will invoke our auth callback for digest
TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
- factory.addServerDefinition(KERBEROS, serviceName, hostName, props, server_callback_handler);
+ factory.addServerDefinition(KERBEROS, serviceName, hostName, props, serverCallbackHandler);
//Also add in support for worker tokens
factory.addServerDefinition(DIGEST, ClientAuthUtils.SERVICE, hostName, null,
@@ -107,11 +107,11 @@
private Login mkLogin() throws IOException {
try {
//create an authentication callback handler
- ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(loginConf);
+ ClientCallbackHandler clientCallbackHandler = new ClientCallbackHandler(loginConf);
//specify a configuration object to be used
Configuration.setConfiguration(loginConf);
//now login
- Login login = new Login(ClientAuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
+ Login login = new Login(ClientAuthUtils.LOGIN_CONTEXT_CLIENT, clientCallbackHandler);
login.startThreadIfNeeded();
return login;
} catch (LoginException ex) {
@@ -204,20 +204,21 @@
//open Sasl transport with the login credential
try {
Subject.doAs(subject,
- new PrivilegedExceptionAction<Void>() {
- @Override
- public Void run() {
- try {
- LOG.debug("do as:" + principal);
- sasalTransport.open();
- } catch (Exception e) {
- LOG.error(
- "Client failed to open SaslClientTransport to interact with a server during session initiation: " +
- e, e);
- }
- return null;
- }
- });
+ new PrivilegedExceptionAction<Void>() {
+ @Override
+ public Void run() {
+ try {
+ LOG.debug("do as:" + principal);
+ sasalTransport.open();
+ } catch (Exception e) {
+ LOG.error("Client failed to open SaslClientTransport to interact with a server during "
+ + "session initiation: "
+ + e,
+ e);
+ }
+ return null;
+ }
+ });
} catch (PrivilegedActionException e) {
throw new RuntimeException(e);
}
@@ -247,8 +248,9 @@
/**
* A TransportFactory that wraps another one, but assumes a specified UGI before calling through.
*
- * This is used on the server side to assume the server's Principal when accepting clients.
+ * <p>This is used on the server side to assume the server's Principal when accepting clients.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
static class TUGIAssumingTransportFactory extends TTransportFactory {
private final Subject subject;
private final TTransportFactory wrapped;
@@ -267,20 +269,22 @@
public TTransport getTransport(final TTransport trans) {
try {
return Subject.doAs(subject,
- (PrivilegedExceptionAction<TTransport>) () -> {
- try {
- return wrapped.getTransport(trans);
- } catch (Exception e) {
- LOG.debug("Storm server failed to open transport " +
- "to interact with a client during session initiation: " + e, e);
- return new NoOpTTrasport(null);
- }
- });
+ (PrivilegedExceptionAction<TTransport>) () -> {
+ try {
+ return wrapped.getTransport(trans);
+ } catch (Exception e) {
+ LOG.debug("Storm server failed to open transport to interact with a client during "
+ + "session initiation: "
+ + e,
+ e);
+ return new NoOpTTrasport(null);
+ }
+ });
} catch (PrivilegedActionException e) {
- LOG.error(
- "Storm server experienced a PrivilegedActionException exception while creating a transport using a JAAS principal " +
- "context:" +
- e, e);
+ LOG.error("Storm server experienced a PrivilegedActionException exception while creating a transport "
+ + "using a JAAS principal context:"
+ + e,
+ e);
return null;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java
index 20595a9..05630e9 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/ServerCallbackHandler.java
@@ -41,7 +41,7 @@
return;
}
- AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(ClientAuthUtils.LOGIN_CONTEXT_SERVER);
+ AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(ClientAuthUtils.LOGIN_CONTEXT_SERVER);
if (configurationEntries == null) {
String errorMessage = "Could not find a '" + ClientAuthUtils.LOGIN_CONTEXT_SERVER
+ "' entry in this configuration: Server cannot start.";
@@ -82,13 +82,13 @@
}
if (ac != null) {
- String authenticationID = ac.getAuthenticationID();
- LOG.debug("Successfully authenticated client: authenticationID={} authorizationID= {}", authenticationID,
+ String authenticationId = ac.getAuthenticationID();
+ LOG.debug("Successfully authenticated client: authenticationID={} authorizationID= {}", authenticationId,
ac.getAuthorizationID());
//if authorizationId is not set, set it to authenticationId.
if (ac.getAuthorizationID() == null) {
- ac.setAuthorizedID(authenticationID);
+ ac.setAuthorizedID(authenticationId);
}
//When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java b/storm-client/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
index de01dc0..697fa5e 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/plain/PlainSaslTransportPlugin.java
@@ -40,7 +40,7 @@
protected TTransportFactory getServerTransportFactory(boolean impersonationAllowed) throws IOException {
//create an authentication callback handler
CallbackHandler serverCallbackHandler = new SimpleSaslServerCallbackHandler(impersonationAllowed,
- (userName) -> Optional.of("password".toCharArray()));
+ (userName) -> Optional.of("password".toCharArray()));
if (Security.getProvider(SaslPlainServer.SecurityProvider.SASL_PLAIN_SERVER) == null) {
Security.addProvider(new SaslPlainServer.SecurityProvider());
}
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/sasl/SaslTransportPlugin.java b/storm-client/src/jvm/org/apache/storm/security/auth/sasl/SaslTransportPlugin.java
index f7cf596..b204c1a 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/sasl/SaslTransportPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/sasl/SaslTransportPlugin.java
@@ -117,6 +117,7 @@
* Processor that pulls the SaslServer object out of the transport, and assumes the remote user's UGI before calling through to the
* original processor. This is used on the server side to set the UGI for each specific call.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static class TUGIWrapProcessor implements TProcessor {
final TProcessor wrapped;
diff --git a/storm-client/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java b/storm-client/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java
index 4c4d4d2..0b993bb 100644
--- a/storm-client/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/security/serialization/BlowfishTupleSerializer.java
@@ -29,14 +29,14 @@
import org.slf4j.LoggerFactory;
/**
- * Apply Blowfish encryption for tuple communication to bolts
+ * Apply Blowfish encryption for tuple communication to bolts.
*/
public class BlowfishTupleSerializer extends Serializer<ListDelegate> {
/**
* The secret key (if any) for data encryption by blowfish payload serialization factory (BlowfishSerializationFactory). You should use
* in via:
*
- * ```storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=org.apache.storm.security.serialization
+ * <p>```storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=org.apache.storm.security.serialization
* .BlowfishTupleSerializer
* jar ...```
*/
@@ -71,7 +71,7 @@
}
/**
- * Produce a blowfish key to be used in "Storm jar" command
+ * Produce a blowfish key to be used in "Storm jar" command.
*/
public static void main(String[] args) {
try {
@@ -80,8 +80,9 @@
SecretKey skey = kgen.generateKey();
byte[] raw = skey.getEncoded();
String keyString = new String(Hex.encodeHex(raw));
- System.out.println("storm -c " + SECRET_KEY + "=" + keyString + " -c " + Config.TOPOLOGY_TUPLE_SERIALIZER + "=" +
- BlowfishTupleSerializer.class.getName() + " ...");
+ System.out.println("storm -c " + SECRET_KEY
+ + "=" + keyString + " -c " + Config.TOPOLOGY_TUPLE_SERIALIZER
+ + "=" + BlowfishTupleSerializer.class.getName() + " ...");
} catch (Exception ex) {
LOG.error(ex.getMessage());
}
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/DefaultKryoFactory.java b/storm-client/src/jvm/org/apache/storm/serialization/DefaultKryoFactory.java
index 336fa45..d361e25 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/DefaultKryoFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/DefaultKryoFactory.java
@@ -42,15 +42,15 @@
}
public static class KryoSerializableDefault extends Kryo {
- boolean _override = false;
+ boolean override = false;
public void overrideDefault(boolean value) {
- _override = value;
+ override = value;
}
@Override
public Serializer getDefaultSerializer(Class type) {
- if (_override) {
+ if (override) {
return new SerializableSerializer();
} else {
return super.getDefaultSerializer(type);
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/GzipBridgeThriftSerializationDelegate.java b/storm-client/src/jvm/org/apache/storm/serialization/GzipBridgeThriftSerializationDelegate.java
index 3e2ec88..bc96611 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/GzipBridgeThriftSerializationDelegate.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/GzipBridgeThriftSerializationDelegate.java
@@ -49,7 +49,7 @@
}
/**
- * Looks ahead to see if the GZIP magic constant is heading {@code bytes}
+ * Looks ahead to see if the GZIP magic constant is heading {@code bytes}.
*/
private boolean isGzipped(byte[] bytes) {
return (bytes.length > 1) && (bytes[0] == GZIP_MAGIC_FIRST_BYTE)
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/IKryoFactory.java b/storm-client/src/jvm/org/apache/storm/serialization/IKryoFactory.java
index daed7e7..3d452b9 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/IKryoFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/IKryoFactory.java
@@ -17,10 +17,14 @@
/**
* An interface that controls the Kryo instance used by Storm for serialization. The lifecycle is:
- *
- * 1. The Kryo instance is constructed using getKryo 2. Storm registers the default classes (e.g. arrays, lists, maps, etc.) 3. Storm calls
- * preRegister hook 4. Storm registers all user-defined registrations through topology.kryo.register 5. Storm calls postRegister hook 6.
- * Storm calls all user-defined decorators through topology.kryo.decorators 7. Storm calls postDecorate hook
+ * <ol><li>The Kryo instance is constructed using getKryo</li>
+ * <li>Storm registers the default classes (e.g. arrays, lists, maps, etc.)</li>
+ * <li>Storm calls preRegister hook</li>
+ * <li>Storm registers all user-defined registrations through topology.kryo.register</li>
+ * <li>Storm calls postRegister hook</li>
+ * <li>Storm calls all user-defined decorators through topology.kryo.decorators</li>
+ * <li>Storm calls postDecorate hook</li>
+ * </ol>
*/
public interface IKryoFactory {
Kryo getKryo(Map<String, Object> conf);
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleDeserializer.java b/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleDeserializer.java
index 128f1c1..0d9f6b9 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleDeserializer.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleDeserializer.java
@@ -21,29 +21,29 @@
import org.apache.storm.tuple.TupleImpl;
public class KryoTupleDeserializer implements ITupleDeserializer {
- GeneralTopologyContext _context;
- KryoValuesDeserializer _kryo;
- SerializationFactory.IdDictionary _ids;
- Input _kryoInput;
+ private GeneralTopologyContext context;
+ private KryoValuesDeserializer kryo;
+ private SerializationFactory.IdDictionary ids;
+ private Input kryoInput;
public KryoTupleDeserializer(final Map<String, Object> conf, final GeneralTopologyContext context) {
- _kryo = new KryoValuesDeserializer(conf);
- _context = context;
- _ids = new SerializationFactory.IdDictionary(context.getRawTopology());
- _kryoInput = new Input(1);
+ kryo = new KryoValuesDeserializer(conf);
+ this.context = context;
+ ids = new SerializationFactory.IdDictionary(context.getRawTopology());
+ kryoInput = new Input(1);
}
@Override
public TupleImpl deserialize(byte[] ser) {
try {
- _kryoInput.setBuffer(ser);
- int taskId = _kryoInput.readInt(true);
- int streamId = _kryoInput.readInt(true);
- String componentName = _context.getComponentId(taskId);
- String streamName = _ids.getStreamName(componentName, streamId);
- MessageId id = MessageId.deserialize(_kryoInput);
- List<Object> values = _kryo.deserializeFrom(_kryoInput);
- return new TupleImpl(_context, values, componentName, taskId, streamName, id);
+ kryoInput.setBuffer(ser);
+ int taskId = kryoInput.readInt(true);
+ int streamId = kryoInput.readInt(true);
+ String componentName = context.getComponentId(taskId);
+ String streamName = ids.getStreamName(componentName, streamId);
+ MessageId id = MessageId.deserialize(kryoInput);
+ List<Object> values = kryo.deserializeFrom(kryoInput);
+ return new TupleImpl(context, values, componentName, taskId, streamName, id);
} catch (IOException e) {
throw new RuntimeException(e);
}
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java b/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java
index ad89438..ae0fc7d 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/KryoTupleSerializer.java
@@ -19,38 +19,28 @@
import org.apache.storm.tuple.Tuple;
public class KryoTupleSerializer implements ITupleSerializer {
- KryoValuesSerializer _kryo;
- SerializationFactory.IdDictionary _ids;
- Output _kryoOut;
+ private KryoValuesSerializer kryo;
+ private SerializationFactory.IdDictionary ids;
+ private Output kryoOut;
public KryoTupleSerializer(final Map<String, Object> conf, final GeneralTopologyContext context) {
- _kryo = new KryoValuesSerializer(conf);
- _kryoOut = new Output(2000, 2000000000);
- _ids = new SerializationFactory.IdDictionary(context.getRawTopology());
+ kryo = new KryoValuesSerializer(conf);
+ kryoOut = new Output(2000, 2000000000);
+ ids = new SerializationFactory.IdDictionary(context.getRawTopology());
}
@Override
public byte[] serialize(Tuple tuple) {
try {
- _kryoOut.clear();
- _kryoOut.writeInt(tuple.getSourceTask(), true);
- _kryoOut.writeInt(_ids.getStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()), true);
- tuple.getMessageId().serialize(_kryoOut);
- _kryo.serializeInto(tuple.getValues(), _kryoOut);
- return _kryoOut.toBytes();
+ kryoOut.clear();
+ kryoOut.writeInt(tuple.getSourceTask(), true);
+ kryoOut.writeInt(ids.getStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()), true);
+ tuple.getMessageId().serialize(kryoOut);
+ kryo.serializeInto(tuple.getValues(), kryoOut);
+ return kryoOut.toBytes();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
-
- // public long crc32(Tuple tuple) {
- // try {
- // CRC32OutputStream hasher = new CRC32OutputStream();
- // _kryo.serializeInto(tuple.getValues(), hasher);
- // return hasher.getValue();
- // } catch (IOException e) {
- // throw new RuntimeException(e);
- // }
- // }
}
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesDeserializer.java b/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesDeserializer.java
index e56b21b..873b064 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesDeserializer.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesDeserializer.java
@@ -19,26 +19,26 @@
import org.apache.storm.utils.ListDelegate;
public class KryoValuesDeserializer {
- Kryo _kryo;
- Input _kryoInput;
+ Kryo kryo;
+ Input kryoInput;
public KryoValuesDeserializer(Map<String, Object> conf) {
- _kryo = SerializationFactory.getKryo(conf);
- _kryoInput = new Input(1);
+ kryo = SerializationFactory.getKryo(conf);
+ kryoInput = new Input(1);
}
public List<Object> deserializeFrom(Input input) {
- ListDelegate delegate = _kryo.readObject(input, ListDelegate.class);
+ ListDelegate delegate = kryo.readObject(input, ListDelegate.class);
return delegate.getDelegate();
}
public List<Object> deserialize(byte[] ser) {
- _kryoInput.setBuffer(ser);
- return deserializeFrom(_kryoInput);
+ kryoInput.setBuffer(ser);
+ return deserializeFrom(kryoInput);
}
public Object deserializeObject(byte[] ser) {
- _kryoInput.setBuffer(ser);
- return _kryo.readClassAndObject(_kryoInput);
+ kryoInput.setBuffer(ser);
+ return kryo.readClassAndObject(kryoInput);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesSerializer.java b/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesSerializer.java
index 5ed2c73..b6af940 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/KryoValuesSerializer.java
@@ -19,14 +19,14 @@
import org.apache.storm.utils.ListDelegate;
public class KryoValuesSerializer {
- Kryo _kryo;
- ListDelegate _delegate;
- Output _kryoOut;
+ Kryo kryo;
+ ListDelegate delegate;
+ Output kryoOut;
public KryoValuesSerializer(Map<String, Object> conf) {
- _kryo = SerializationFactory.getKryo(conf);
- _delegate = new ListDelegate();
- _kryoOut = new Output(2000, 2000000000);
+ kryo = SerializationFactory.getKryo(conf);
+ delegate = new ListDelegate();
+ kryoOut = new Output(2000, 2000000000);
}
public void serializeInto(List<Object> values, Output out) {
@@ -34,19 +34,19 @@
// of whether it's a java collection or one of clojure's persistent collections
// (which have different serializers)
// Doing this lets us deserialize as ArrayList and avoid writing the class here
- _delegate.setDelegate(values);
- _kryo.writeObject(out, _delegate);
+ delegate.setDelegate(values);
+ kryo.writeObject(out, delegate);
}
public byte[] serialize(List<Object> values) {
- _kryoOut.clear();
- serializeInto(values, _kryoOut);
- return _kryoOut.toBytes();
+ kryoOut.clear();
+ serializeInto(values, kryoOut);
+ return kryoOut.toBytes();
}
public byte[] serializeObject(Object obj) {
- _kryoOut.clear();
- _kryo.writeClassAndObject(_kryoOut, obj);
- return _kryoOut.toBytes();
+ kryoOut.clear();
+ kryo.writeClassAndObject(kryoOut, obj);
+ return kryoOut.toBytes();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java b/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
index ef9b75c..431746f 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/SerializationFactory.java
@@ -224,9 +224,9 @@
}
/**
- * "{:a 1 :b 2} -> {1 :a 2 :b}"
+ * "{:a 1 :b 2} -> {1 :a 2 :b}".
*
- * Note: Only one key wins if there are duplicate values. Which key wins is indeterminate: "{:a 1 :b 1} -> {1 :a} *or* {1 :b}"
+ * <p>Note: Only one key wins if there are duplicate values. Which key wins is indeterminate: "{:a 1 :b 1} -> {1 :a} *or* {1 :b}"
*/
private static <K, V> Map<V, K> simpleReverseMap(Map<K, V> map) {
Map<V, K> ret = new HashMap<V, K>();
diff --git a/storm-client/src/jvm/org/apache/storm/serialization/SerializationRegister.java b/storm-client/src/jvm/org/apache/storm/serialization/SerializationRegister.java
index 4bec09f..06954ce 100644
--- a/storm-client/src/jvm/org/apache/storm/serialization/SerializationRegister.java
+++ b/storm-client/src/jvm/org/apache/storm/serialization/SerializationRegister.java
@@ -26,7 +26,7 @@
*/
public interface SerializationRegister {
/**
- * Register any serializers needed with the kryo instance
+ * Register any serializers needed with the kryo instance.
*
* @param kryo what to register the serializers with.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/spout/CheckPointState.java b/storm-client/src/jvm/org/apache/storm/spout/CheckPointState.java
index 49f140d..7b327da 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/CheckPointState.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/CheckPointState.java
@@ -25,13 +25,12 @@
*
* </pre>
*
- * During recovery, if a previous transaction is in PREPARING state, it is rolled back since all bolts in the topology might not have
+ * <p>During recovery, if a previous transaction is in PREPARING state, it is rolled back since all bolts in the topology might not have
* prepared (saved) the data for commit. If the previous transaction is in COMMITTING state, it is rolled forward (committed) since some
* bolts might have already committed the data.
- * <p>
- * During normal flow, the state transitions from PREPARING to COMMITTING to COMMITTED. In case of failures the prepare/commit operation is
- * retried.
- * </p>
+ *
+ * <p>During normal flow, the state transitions from PREPARING to COMMITTING to COMMITTED. In case of failures the
+ * prepare/commit operation is retried.
*/
public class CheckPointState {
private long txid;
@@ -129,10 +128,10 @@
@Override
public String toString() {
- return "CheckPointState{" +
- "txid=" + txid +
- ", state=" + state +
- '}';
+ return "CheckPointState{"
+ + "txid=" + txid
+ + ", state=" + state
+ + '}';
}
public enum State {
@@ -152,19 +151,19 @@
public enum Action {
/**
- * prepare transaction for commit
+ * prepare transaction for commit.
*/
PREPARE,
/**
- * commit the previously prepared transaction
+ * commit the previously prepared transaction.
*/
COMMIT,
/**
- * rollback the previously prepared transaction
+ * rollback the previously prepared transaction.
*/
ROLLBACK,
/**
- * initialize the state
+ * initialize the state.
*/
INITSTATE
}
diff --git a/storm-client/src/jvm/org/apache/storm/spout/CheckpointSpout.java b/storm-client/src/jvm/org/apache/storm/spout/CheckpointSpout.java
index 6d79142..c6f6fe4 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/CheckpointSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/CheckpointSpout.java
@@ -18,6 +18,9 @@
package org.apache.storm.spout;
+import static org.apache.storm.spout.CheckPointState.Action;
+import static org.apache.storm.spout.CheckPointState.State.COMMITTED;
+
import java.util.Map;
import org.apache.storm.Config;
import org.apache.storm.state.KeyValueState;
@@ -32,9 +35,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.spout.CheckPointState.Action;
-import static org.apache.storm.spout.CheckPointState.State.COMMITTED;
-
/**
* Emits checkpoint tuples which is used to save the state of the {@link org.apache.storm.topology.IStatefulComponent} across the topology.
* If a topology contains Stateful bolts, Checkpoint spouts are automatically added to the topology. There is only one Checkpoint task per
@@ -161,8 +161,9 @@
}
private boolean shouldCheckpoint() {
- return !recovering && !checkpointStepInProgress &&
- (curTxState.getState() != COMMITTED || checkpointIntervalElapsed());
+ return !recovering
+ && !checkpointStepInProgress
+ && (curTxState.getState() != COMMITTED || checkpointIntervalElapsed());
}
private boolean checkpointIntervalElapsed() {
diff --git a/storm-client/src/jvm/org/apache/storm/spout/ISpout.java b/storm-client/src/jvm/org/apache/storm/spout/ISpout.java
index ff97e31..120ab36 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/ISpout.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/ISpout.java
@@ -17,27 +17,30 @@
import org.apache.storm.task.TopologyContext;
/**
- * ISpout is the core interface for implementing spouts. A Spout is responsible for feeding messages into the topology for processing. For
- * every tuple emitted by a spout, Storm will track the (potentially very large) DAG of tuples generated based on a tuple emitted by the
- * spout. When Storm detects that every tuple in that DAG has been successfully processed, it will send an ack message to the Spout.
+ * ISpout is the core interface for implementing spouts. A Spout is responsible for feeding messages into the topology
+ * for processing. For every tuple emitted by a spout, Storm will track the (potentially very large) DAG of tuples
+ * generated based on a tuple emitted by the spout. When Storm detects that every tuple in that DAG has been
+ * successfully processed, it will send an ack message to the Spout.
*
- * If a tuple fails to be fully processed within the configured timeout for the topology (see {@link org.apache.storm.Config}), Storm will
- * send a fail message to the spout for the message.
+ * <p>If a tuple fails to be fully processed within the configured timeout for the topology (see
+ * {@link org.apache.storm.Config}), Storm will send a fail message to the spout for the message.
*
- * When a Spout emits a tuple, it can tag the tuple with a message id. The message id can be any type. When Storm acks or fails a message,
- * it will pass back to the spout the same message id to identify which tuple it's referring to. If the spout leaves out the message id, or
- * sets it to null, then Storm will not track the message and the spout will not receive any ack or fail callbacks for the message.
+ * <p>When a Spout emits a tuple, it can tag the tuple with a message id. The message id can be any type. When Storm
+ * acks or fails a message, it will pass back to the spout the same message id to identify which tuple it's referring
+ * to. If the spout leaves out the message id, or sets it to null, then Storm will not track the message and the spout
+ * will not receive any ack or fail callbacks for the message.
*
- * Storm executes ack, fail, and nextTuple all on the same thread. This means that an implementor of an ISpout does not need to worry about
- * concurrency issues between those methods. However, it also means that an implementor must ensure that nextTuple is non-blocking:
- * otherwise the method could block acks and fails that are pending to be processed.
+ * <p>Storm executes ack, fail, and nextTuple all on the same thread. This means that an implementor of an ISpout does
+ * not need to worry about concurrency issues between those methods. However, it also means that an implementor must
+ * ensure that nextTuple is non-blocking: otherwise the method could block acks and fails that are pending to be
+ * processed.
*/
public interface ISpout extends Serializable {
/**
* Called when a task for this component is initialized within a worker on the cluster. It provides the spout with the environment in
* which the spout executes.
*
- * This includes the:
+ * <p>This includes the:
*
* @param conf The Storm configuration for this spout. This is the configuration provided to the topology merged in with cluster
* configuration on this machine.
@@ -52,7 +55,7 @@
* Called when an ISpout is going to be shutdown. There is no guarentee that close will be called, because the supervisor kill -9's
* worker processes on the cluster.
*
- * The one context where close is guaranteed to be called is a topology is killed when running Storm in local mode.
+ * <p>The one context where close is guaranteed to be called is a topology is killed when running Storm in local mode.
*/
void close();
diff --git a/storm-client/src/jvm/org/apache/storm/spout/RawMultiScheme.java b/storm-client/src/jvm/org/apache/storm/spout/RawMultiScheme.java
index c762358..14c6565 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/RawMultiScheme.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/RawMultiScheme.java
@@ -18,13 +18,13 @@
package org.apache.storm.spout;
+import static java.util.Arrays.asList;
+
import java.nio.ByteBuffer;
import java.util.List;
import org.apache.storm.tuple.Fields;
import org.apache.storm.utils.Utils;
-import static java.util.Arrays.asList;
-
public class RawMultiScheme implements MultiScheme {
@Override
public Iterable<List<Object>> deserialize(ByteBuffer ser) {
diff --git a/storm-client/src/jvm/org/apache/storm/spout/ShellSpout.java b/storm-client/src/jvm/org/apache/storm/spout/ShellSpout.java
index 317aec1..5312909 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/ShellSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/ShellSpout.java
@@ -42,17 +42,17 @@
public static final Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
private static final long serialVersionUID = 5982357019665454L;
- private SpoutOutputCollector _collector;
- private String[] _command;
+ private SpoutOutputCollector collector;
+ private String[] command;
private Map<String, String> env = new HashMap<>();
- private ShellLogHandler _logHandler;
- private ShellProcess _process;
- private volatile boolean _running = true;
- private volatile RuntimeException _exception;
+ private ShellLogHandler logHandler;
+ private ShellProcess process;
+ private volatile boolean running = true;
+ private volatile RuntimeException exception;
- private TopologyContext _context;
+ private TopologyContext context;
- private SpoutMsg _spoutMsg;
+ private SpoutMsg spoutMsg;
private int workerTimeoutMills;
private ScheduledExecutorService heartBeatExecutorService;
@@ -65,7 +65,7 @@
}
public ShellSpout(String... command) {
- _command = command;
+ this.command = command;
}
public ShellSpout setEnv(Map<String, String> env) {
@@ -73,10 +73,12 @@
return this;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public boolean shouldChangeChildCWD() {
return changeDirectory;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
/**
* Set if the current working directory of the child process should change to the resources dir from extracted from the jar, or if it
* should stay the same as the worker process to access things from the blob store.
@@ -90,8 +92,8 @@
@Override
public void open(Map<String, Object> topoConf, TopologyContext context,
SpoutOutputCollector collector) {
- _collector = collector;
- _context = context;
+ this.collector = collector;
+ this.context = context;
if (topoConf.containsKey(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS)) {
workerTimeoutMills = 1000 * ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS));
@@ -99,16 +101,16 @@
workerTimeoutMills = 1000 * ObjectReader.getInt(topoConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
}
- _process = new ShellProcess(_command);
+ process = new ShellProcess(command);
if (!env.isEmpty()) {
- _process.setEnv(env);
+ process.setEnv(env);
}
- Number subpid = _process.launch(topoConf, context, changeDirectory);
+ Number subpid = process.launch(topoConf, context, changeDirectory);
LOG.info("Launched subprocess with pid " + subpid);
- _logHandler = ShellUtils.getLogHandler(topoConf);
- _logHandler.setUpContext(ShellSpout.class, _process, _context);
+ logHandler = ShellUtils.getLogHandler(topoConf);
+ logHandler.setUpContext(ShellSpout.class, process, this.context);
heartBeatExecutorService = MoreExecutors.getExitingScheduledExecutorService(new ScheduledThreadPoolExecutor(1));
}
@@ -116,8 +118,8 @@
@Override
public void close() {
heartBeatExecutorService.shutdownNow();
- _process.destroy();
- _running = false;
+ process.destroy();
+ running = false;
}
@Override
@@ -136,15 +138,15 @@
}
private void sendSyncCommand(String command, Object msgId) {
- if (_exception != null) {
- throw _exception;
+ if (exception != null) {
+ throw exception;
}
- if (_spoutMsg == null) {
- _spoutMsg = new SpoutMsg();
+ if (spoutMsg == null) {
+ spoutMsg = new SpoutMsg();
}
- _spoutMsg.setCommand(command);
- _spoutMsg.setId(msgId);
+ spoutMsg.setCommand(command);
+ spoutMsg.setId(msgId);
querySubprocess();
}
@@ -157,19 +159,19 @@
}
//get metric by name
- IMetric iMetric = _context.getRegisteredMetricByName(name);
- if (iMetric == null) {
+ IMetric metric = context.getRegisteredMetricByName(name);
+ if (metric == null) {
throw new RuntimeException("Could not find metric by name[" + name + "] ");
}
- if (!(iMetric instanceof IShellMetric)) {
+ if (!(metric instanceof IShellMetric)) {
throw new RuntimeException("Metric[" + name + "] is not IShellMetric, can not call by RPC");
}
- IShellMetric iShellMetric = (IShellMetric) iMetric;
+ IShellMetric shellMetric = (IShellMetric) metric;
//call updateMetricFromRPC with params
Object paramsObj = shellMsg.getMetricParams();
try {
- iShellMetric.updateMetricFromRPC(paramsObj);
+ shellMetric.updateMetricFromRPC(paramsObj);
} catch (RuntimeException re) {
throw re;
} catch (Exception e) {
@@ -180,10 +182,10 @@
private void querySubprocess() {
try {
markWaitingSubprocess();
- _process.writeSpoutMsg(_spoutMsg);
+ process.writeSpoutMsg(spoutMsg);
while (true) {
- ShellMsg shellMsg = _process.readShellMsg();
+ ShellMsg shellMsg = process.readShellMsg();
String command = shellMsg.getCommand();
if (command == null) {
throw new IllegalArgumentException("Command not found in spout message: " + shellMsg);
@@ -194,7 +196,7 @@
if (command.equals("sync")) {
return;
} else if (command.equals("log")) {
- _logHandler.log(shellMsg);
+ logHandler.log(shellMsg);
} else if (command.equals("error")) {
handleError(shellMsg.getMsg());
} else if (command.equals("emit")) {
@@ -203,12 +205,12 @@
List<Object> tuple = shellMsg.getTuple();
Object messageId = shellMsg.getId();
if (task == 0) {
- List<Integer> outtasks = _collector.emit(stream, tuple, messageId);
+ List<Integer> outtasks = collector.emit(stream, tuple, messageId);
if (shellMsg.areTaskIdsNeeded()) {
- _process.writeTaskIds(outtasks);
+ process.writeTaskIds(outtasks);
}
} else {
- _collector.emitDirect((int) task.longValue(), stream, tuple, messageId);
+ collector.emitDirect((int) task.longValue(), stream, tuple, messageId);
}
} else if (command.equals("metrics")) {
handleMetrics(shellMsg);
@@ -217,7 +219,7 @@
}
}
} catch (Exception e) {
- String processInfo = _process.getProcessInfoString() + _process.getProcessTerminationInfoString();
+ String processInfo = process.getProcessInfoString() + process.getProcessTerminationInfoString();
throw new RuntimeException(processInfo, e);
} finally {
completedWaitingSubprocess();
@@ -225,7 +227,7 @@
}
private void handleError(String msg) {
- _collector.reportError(new Exception("Shell Process Exception: " + msg));
+ collector.reportError(new Exception("Shell Process Exception: " + msg));
}
@Override
@@ -265,14 +267,14 @@
}
private void die(Throwable exception) {
- String processInfo = _process.getProcessInfoString() + _process.getProcessTerminationInfoString();
- _exception = new RuntimeException(processInfo, exception);
+ String processInfo = process.getProcessInfoString() + process.getProcessTerminationInfoString();
+ this.exception = new RuntimeException(processInfo, exception);
String message = String.format("Halting process: ShellSpout died. Command: %s, ProcessInfo %s",
- Arrays.toString(_command),
+ Arrays.toString(command),
processInfo);
LOG.error(message, exception);
- _collector.reportError(exception);
- if (_running || (exception instanceof Error)) { //don't exit if not running, unless it is an Error
+ collector.reportError(exception);
+ if (running || (exception instanceof Error)) { //don't exit if not running, unless it is an Error
System.exit(11);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java b/storm-client/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java
index f8895e6..09f155d 100644
--- a/storm-client/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/spout/SpoutOutputCollector.java
@@ -23,10 +23,10 @@
* processed at least once.
*/
public class SpoutOutputCollector implements ISpoutOutputCollector {
- ISpoutOutputCollector _delegate;
+ ISpoutOutputCollector delegate;
public SpoutOutputCollector(ISpoutOutputCollector delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
/**
@@ -40,7 +40,7 @@
*/
@Override
public List<Integer> emit(String streamId, List<Object> tuple, Object messageId) {
- return _delegate.emit(streamId, tuple, messageId);
+ return delegate.emit(streamId, tuple, messageId);
}
/**
@@ -53,7 +53,7 @@
* @return the list of task ids that this tuple was sent to
*/
public List<Integer> emit(List<Object> tuple, Object messageId) {
- return _delegate.emit(Utils.DEFAULT_STREAM_ID, tuple, messageId);
+ return delegate.emit(Utils.DEFAULT_STREAM_ID, tuple, messageId);
}
/**
@@ -80,7 +80,7 @@
*/
@Override
public void emitDirect(int taskId, String streamId, List<Object> tuple, Object messageId) {
- _delegate.emitDirect(taskId, streamId, tuple, messageId);
+ delegate.emitDirect(taskId, streamId, tuple, messageId);
}
/**
@@ -96,7 +96,7 @@
* Emits a tuple to the specified task on the specified output stream. This output stream must have been declared as a direct stream,
* and the specified task must use a direct grouping on this stream to receive the message. The emitted values must be immutable.
*
- * Because no message id is specified, Storm will not track this message so ack and fail will never be called for this tuple.
+ * <p>Because no message id is specified, Storm will not track this message so ack and fail will never be called for this tuple.
*/
public void emitDirect(int taskId, String streamId, List<Object> tuple) {
emitDirect(taskId, streamId, tuple, null);
@@ -106,7 +106,7 @@
* Emits a tuple to the specified task on the default output stream. This output stream must have been declared as a direct stream, and
* the specified task must use a direct grouping on this stream to receive the message. The emitted values must be immutable.
*
- * Because no message id is specified, Storm will not track this message so ack and fail will never be called for this tuple.
+ * <p>Because no message id is specified, Storm will not track this message so ack and fail will never be called for this tuple.
*/
public void emitDirect(int taskId, List<Object> tuple) {
emitDirect(taskId, tuple, null);
@@ -114,16 +114,16 @@
@Override
public void flush() {
- _delegate.flush();
+ delegate.flush();
}
@Override
public void reportError(Throwable error) {
- _delegate.reportError(error);
+ delegate.reportError(error);
}
@Override
public long getPendingCount() {
- return _delegate.getPendingCount();
+ return delegate.getPendingCount();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/state/BaseStateIterator.java b/storm-client/src/jvm/org/apache/storm/state/BaseStateIterator.java
index 95caabb..4af027f 100644
--- a/storm-client/src/jvm/org/apache/storm/state/BaseStateIterator.java
+++ b/storm-client/src/jvm/org/apache/storm/state/BaseStateIterator.java
@@ -23,15 +23,15 @@
/**
* Base implementation of iterator over {@link KeyValueState}. Encoded/Decoded types of key and value are all generic.
*/
-public abstract class BaseStateIterator<K, V, KENCODED, VENCODED> implements Iterator<Map.Entry<K, V>> {
+public abstract class BaseStateIterator<K, V, KENCODEDT, VENCODEDT> implements Iterator<Map.Entry<K, V>> {
- private final PeekingIterator<Map.Entry<KENCODED, VENCODED>> pendingPrepareIterator;
- private final PeekingIterator<Map.Entry<KENCODED, VENCODED>> pendingCommitIterator;
- private final Set<KENCODED> providedKeys;
+ private final PeekingIterator<Map.Entry<KENCODEDT, VENCODEDT>> pendingPrepareIterator;
+ private final PeekingIterator<Map.Entry<KENCODEDT, VENCODEDT>> pendingCommitIterator;
+ private final Set<KENCODEDT> providedKeys;
private boolean firstLoad = true;
- private PeekingIterator<Map.Entry<KENCODED, VENCODED>> pendingIterator;
- private PeekingIterator<Map.Entry<KENCODED, VENCODED>> cachedResultIterator;
+ private PeekingIterator<Map.Entry<KENCODEDT, VENCODEDT>> pendingIterator;
+ private PeekingIterator<Map.Entry<KENCODEDT, VENCODEDT>> cachedResultIterator;
/**
* Constructor.
@@ -40,9 +40,9 @@
* @param pendingCommitIterator The iterator of pendingCommit
* @param initialProvidedKeys The initial value of provided keys
*/
- public BaseStateIterator(Iterator<Map.Entry<KENCODED, VENCODED>> pendingPrepareIterator,
- Iterator<Map.Entry<KENCODED, VENCODED>> pendingCommitIterator,
- Set<KENCODED> initialProvidedKeys) {
+ public BaseStateIterator(Iterator<Map.Entry<KENCODEDT, VENCODEDT>> pendingPrepareIterator,
+ Iterator<Map.Entry<KENCODEDT, VENCODEDT>> pendingCommitIterator,
+ Set<KENCODEDT> initialProvidedKeys) {
this.pendingPrepareIterator = Iterators.peekingIterator(pendingPrepareIterator);
this.pendingCommitIterator = Iterators.peekingIterator(pendingCommitIterator);
this.providedKeys = initialProvidedKeys;
@@ -85,7 +85,7 @@
}
private void fillCachedResultIterator() {
- Iterator<Map.Entry<KENCODED, VENCODED>> iterator = loadChunkFromStateStorage();
+ Iterator<Map.Entry<KENCODEDT, VENCODEDT>> iterator = loadChunkFromStateStorage();
if (iterator != null) {
cachedResultIterator = Iterators.peekingIterator(iterator);
} else {
@@ -99,7 +99,7 @@
throw new NoSuchElementException();
}
- Map.Entry<KENCODED, VENCODED> keyValue = pendingIterator.next();
+ Map.Entry<KENCODEDT, VENCODEDT> keyValue = pendingIterator.next();
K key = decodeKey(keyValue.getKey());
V value = decodeValue(keyValue.getValue());
@@ -118,7 +118,7 @@
*
* @return Iterator of loaded state KVs
*/
- protected abstract Iterator<Map.Entry<KENCODED, VENCODED>> loadChunkFromStateStorage();
+ protected abstract Iterator<Map.Entry<KENCODEDT, VENCODEDT>> loadChunkFromStateStorage();
/**
* Check whether end of data is reached from storage state KVs.
@@ -133,7 +133,7 @@
* @param key raw type of encoded key
* @return Decoded value of key
*/
- protected abstract K decodeKey(KENCODED key);
+ protected abstract K decodeKey(KENCODEDT key);
/**
* Decode value to convert encoded type of value to state value type.
@@ -141,7 +141,7 @@
* @param value raw type of encoded value
* @return Decoded value of value
*/
- protected abstract V decodeValue(VENCODED value);
+ protected abstract V decodeValue(VENCODEDT value);
/**
* Check whether the value is tombstone (deletion mark) value.
@@ -149,12 +149,12 @@
* @param value the value to check
* @return true if the value is tombstone, false otherwise
*/
- protected abstract boolean isTombstoneValue(VENCODED value);
+ protected abstract boolean isTombstoneValue(VENCODEDT value);
- private boolean seekToAvailableEntry(PeekingIterator<Map.Entry<KENCODED, VENCODED>> iterator) {
+ private boolean seekToAvailableEntry(PeekingIterator<Map.Entry<KENCODEDT, VENCODEDT>> iterator) {
if (iterator != null) {
while (iterator.hasNext()) {
- Map.Entry<KENCODED, VENCODED> entry = iterator.peek();
+ Map.Entry<KENCODEDT, VENCODEDT> entry = iterator.peek();
if (!providedKeys.contains(entry.getKey())) {
if (isTombstoneValue(entry.getValue())) {
providedKeys.add(entry.getKey());
diff --git a/storm-client/src/jvm/org/apache/storm/state/DefaultStateEncoder.java b/storm-client/src/jvm/org/apache/storm/state/DefaultStateEncoder.java
index 42a9a4d..48afe47 100644
--- a/storm-client/src/jvm/org/apache/storm/state/DefaultStateEncoder.java
+++ b/storm-client/src/jvm/org/apache/storm/state/DefaultStateEncoder.java
@@ -12,7 +12,6 @@
package org.apache.storm.state;
-
import java.util.Optional;
/**
@@ -21,9 +20,9 @@
*/
public class DefaultStateEncoder<K, V> implements StateEncoder<K, V, byte[], byte[]> {
- public static final Serializer<Optional<byte[]>> internalValueSerializer = new DefaultStateSerializer<>();
+ public static final Serializer<Optional<byte[]>> INTERNAL_VALUE_SERIALIZER = new DefaultStateSerializer<>();
- public static final byte[] TOMBSTONE = internalValueSerializer.serialize(Optional.<byte[]>empty());
+ public static final byte[] TOMBSTONE = INTERNAL_VALUE_SERIALIZER.serialize(Optional.<byte[]>empty());
private final Serializer<K> keySerializer;
private final Serializer<V> valueSerializer;
@@ -48,7 +47,7 @@
@Override
public byte[] encodeValue(V value) {
- return internalValueSerializer.serialize(
+ return INTERNAL_VALUE_SERIALIZER.serialize(
Optional.of(valueSerializer.serialize(value)));
}
@@ -59,7 +58,7 @@
@Override
public V decodeValue(byte[] encodedValue) {
- Optional<byte[]> internalValue = internalValueSerializer.deserialize(encodedValue);
+ Optional<byte[]> internalValue = INTERNAL_VALUE_SERIALIZER.deserialize(encodedValue);
if (internalValue.isPresent()) {
return valueSerializer.deserialize(internalValue.get());
}
diff --git a/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java b/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
index 9425453..c58bb27 100644
--- a/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
+++ b/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueState.java
@@ -19,7 +19,7 @@
import org.slf4j.LoggerFactory;
/**
- * An in-memory implementation of the {@link State}
+ * An in-memory implementation of the {@link State}.
*/
public class InMemoryKeyValueState<K, V> implements KeyValueState<K, V> {
private static final Logger LOG = LoggerFactory.getLogger(InMemoryKeyValueState.class);
@@ -60,27 +60,27 @@
}
@Override
- public void prepareCommit(long txid) {
- LOG.debug("prepare commit, txid {}", txid);
- if (preparedState != null && txid > preparedState.txid) {
- throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
- }
- preparedState = new TxIdState<>(txid, new ConcurrentHashMap<K, V>(state));
- }
-
- @Override
public void commit(long txid) {
LOG.debug("commit, txid {}", txid);
if (preparedState != null && txid == preparedState.txid) {
commitedState = preparedState;
preparedState = null;
} else {
- throw new RuntimeException("Invalid prepared state for commit, " +
- "preparedState " + preparedState + " txid " + txid);
+ throw new RuntimeException("Invalid prepared state for commit, "
+ + "preparedState " + preparedState + " txid " + txid);
}
}
@Override
+ public void prepareCommit(long txid) {
+ LOG.debug("prepare commit, txid {}", txid);
+ if (preparedState != null && txid > preparedState.txid) {
+ throw new RuntimeException("Cannot prepare a new txn while there is a pending txn");
+ }
+ preparedState = new TxIdState<>(txid, new ConcurrentHashMap<K, V>(state));
+ }
+
+ @Override
public void rollback() {
preparedState = null;
if (commitedState != null) {
@@ -92,11 +92,11 @@
@Override
public String toString() {
- return "InMemoryKeyValueState{" +
- "commitedState=" + commitedState +
- ", preparedState=" + preparedState +
- ", state=" + state +
- '}';
+ return "InMemoryKeyValueState{"
+ + "commitedState=" + commitedState
+ + ", preparedState=" + preparedState
+ + ", state=" + state
+ + '}';
}
private static class TxIdState<K, V> {
@@ -110,10 +110,10 @@
@Override
public String toString() {
- return "TxIdState{" +
- "txid=" + txid +
- ", state=" + state +
- '}';
+ return "TxIdState{"
+ + "txid=" + txid
+ + ", state=" + state
+ + '}';
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java b/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
index 0fe6a3a..8cbc617 100644
--- a/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
+++ b/storm-client/src/jvm/org/apache/storm/state/InMemoryKeyValueStateProvider.java
@@ -17,7 +17,7 @@
import org.apache.storm.task.TopologyContext;
/**
- * Provides {@link InMemoryKeyValueState}
+ * Provides {@link InMemoryKeyValueState}.
*/
public class InMemoryKeyValueStateProvider implements StateProvider {
private final ConcurrentHashMap<String, State> states = new ConcurrentHashMap<>();
diff --git a/storm-client/src/jvm/org/apache/storm/state/KeyValueState.java b/storm-client/src/jvm/org/apache/storm/state/KeyValueState.java
index 30d40fd..a8e2e68 100644
--- a/storm-client/src/jvm/org/apache/storm/state/KeyValueState.java
+++ b/storm-client/src/jvm/org/apache/storm/state/KeyValueState.java
@@ -19,7 +19,7 @@
*/
public interface KeyValueState<K, V> extends State, Iterable<Map.Entry<K, V>> {
/**
- * Maps the value with the key
+ * Maps the value with the key.
*
* @param key the key
* @param value the value
@@ -27,7 +27,7 @@
void put(K key, V value);
/**
- * Returns the value mapped to the key
+ * Returns the value mapped to the key.
*
* @param key the key
* @return the value or null if no mapping is found
@@ -44,7 +44,7 @@
V get(K key, V defaultValue);
/**
- * Deletes the value mapped to the key, if there is any
+ * Deletes the value mapped to the key, if there is any.
*
* @param key the key
*/
diff --git a/storm-client/src/jvm/org/apache/storm/state/StateEncoder.java b/storm-client/src/jvm/org/apache/storm/state/StateEncoder.java
index 42f3ef5..687bd5e 100644
--- a/storm-client/src/jvm/org/apache/storm/state/StateEncoder.java
+++ b/storm-client/src/jvm/org/apache/storm/state/StateEncoder.java
@@ -15,22 +15,22 @@
/**
* The interface of State Encoder.
*/
-public interface StateEncoder<K, V, KENCODED, VENCODED> {
+public interface StateEncoder<K, V, KENCODEDT, VENCODEDT> {
/**
* Encode key.
*
* @param key the value of key (K type)
- * @return the encoded value of key (KENCODED type)
+ * @return the encoded value of key (KENCODEDT type)
*/
- KENCODED encodeKey(K key);
+ KENCODEDT encodeKey(K key);
/**
* Encode value.
*
* @param value the value of value (V type)
- * @return the encoded value of value (VENCODED type)
+ * @return the encoded value of value (VENCODEDT type)
*/
- VENCODED encodeValue(V value);
+ VENCODEDT encodeValue(V value);
/**
* Decode key.
@@ -38,20 +38,20 @@
* @param encodedKey the value of key (KRAW type)
* @return the decoded value of key (K type)
*/
- K decodeKey(KENCODED encodedKey);
+ K decodeKey(KENCODEDT encodedKey);
/**
* Decode value.
*
- * @param encodedValue the value of key (VENCODED type)
+ * @param encodedValue the value of key (VENCODEDT type)
* @return the decoded value of key (V type)
*/
- V decodeValue(VENCODED encodedValue);
+ V decodeValue(VENCODEDT encodedValue);
/**
* Get the tombstone value (deletion mark).
*
- * @return the tomestone value (VENCODED type)
+ * @return the tomestone value (VENCODEDT type)
*/
- VENCODED getTombstoneValue();
+ VENCODEDT getTombstoneValue();
}
diff --git a/storm-client/src/jvm/org/apache/storm/state/StateFactory.java b/storm-client/src/jvm/org/apache/storm/state/StateFactory.java
index 7ad052e..4f25402 100644
--- a/storm-client/src/jvm/org/apache/storm/state/StateFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/state/StateFactory.java
@@ -19,7 +19,7 @@
import org.slf4j.LoggerFactory;
/**
- * A factory for creating {@link State} instances
+ * A factory for creating {@link State} instances.
*/
public class StateFactory {
private static final Logger LOG = LoggerFactory.getLogger(StateFactory.class);
@@ -49,8 +49,8 @@
if (object instanceof StateProvider) {
state = ((StateProvider) object).newState(namespace, topoConf, context);
} else {
- String msg = "Invalid state provider '" + provider +
- "'. Should implement org.apache.storm.state.StateProvider";
+ String msg = "Invalid state provider '" + provider
+ + "'. Should implement org.apache.storm.state.StateProvider";
LOG.error(msg);
throw new RuntimeException(msg);
}
diff --git a/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
index db2f579..62269a4 100644
--- a/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
+++ b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
@@ -42,7 +42,7 @@
public static final IdentityTransformer IDENTITY = new IdentityTransformer();
/**
- * Convert a List<Long> executor to java List<Integer>.
+ * Convert a List<Long> executor to java List<Integer>.
*/
public static List<Integer> convertExecutor(List<Long> executor) {
return Lists.newArrayList(executor.get(0).intValue(), executor.get(1).intValue());
@@ -50,8 +50,8 @@
/**
* Make an map of executors to empty stats, in preparation for doing a heartbeat.
- * @param executors the executors as keys of the map.
- * @return and empty map of executors to stats.
+ * @param executors the executors as keys of the map
+ * @return and empty map of executors to stats
*/
public static Map<List<Integer>, ExecutorStats> mkEmptyExecutorZkHbs(Set<List<Long>> executors) {
Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
@@ -77,8 +77,8 @@
* Create a new worker heartbeat for zookeeper.
* @param topoId the topology id
* @param executorStats the stats for the executors
- * @param uptime the uptime for the worker.
- * @return the heartbeat map.
+ * @param uptime the uptime for the worker
+ * @return the heartbeat map
*/
public static Map<String, Object> mkZkWorkerHb(String topoId, Map<List<Integer>, ExecutorStats> executorStats, Integer uptime) {
Map<String, Object> ret = new HashMap<>();
@@ -105,8 +105,8 @@
/**
* Get a sub-map by a given key.
* @param map the original map
- * @param key the key to get it from.
- * @return the map stored under key.
+ * @param key the key to get it from
+ * @return the map stored under key
*/
public static <K, V> Map<K, V> getMapByKey(Map map, String key) {
if (map == null) {
@@ -174,7 +174,6 @@
/**
* Provides a way to transform one key into another.
- * @param <T>
*/
interface KeyTransformer<T> {
T transform(Object key);
diff --git a/storm-client/src/jvm/org/apache/storm/streams/Edge.java b/storm-client/src/jvm/org/apache/storm/streams/Edge.java
index 556645a..168918c 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/Edge.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/Edge.java
@@ -15,7 +15,7 @@
import java.io.Serializable;
/**
- * An edge connects source and target nodes
+ * An edge connects source and target nodes.
*/
class Edge implements Serializable {
private final Node source;
diff --git a/storm-client/src/jvm/org/apache/storm/streams/GroupingInfo.java b/storm-client/src/jvm/org/apache/storm/streams/GroupingInfo.java
index 3e94120..b98e729 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/GroupingInfo.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/GroupingInfo.java
@@ -91,8 +91,8 @@
@Override
public String toString() {
- return "GroupingInfo{" +
- "fields=" + fields +
- '}';
+ return "GroupingInfo{"
+ + "fields=" + fields
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/Node.java b/storm-client/src/jvm/org/apache/storm/streams/Node.java
index 51b5b27..9857536 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/Node.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/Node.java
@@ -75,6 +75,10 @@
return new Fields();
}
+ public Fields getOutputFields() {
+ return outputFields;
+ }
+
GroupingInfo getGroupingInfo() {
return groupingInfo;
}
@@ -83,10 +87,6 @@
this.groupingInfo = groupingInfo;
}
- public Fields getOutputFields() {
- return outputFields;
- }
-
String getComponentId() {
return componentId;
}
@@ -150,15 +150,15 @@
@Override
public String toString() {
- return "Node{" +
- "outputStreams=" + outputStreams +
- ", outputFields=" + outputFields +
- ", groupingInfo=" + groupingInfo +
- ", componentId='" + componentId + '\'' +
- ", parallelism=" + parallelism +
- ", parentStreams=" + parentStreams +
- ", windowed=" + windowed +
- ", emitsPair=" + emitsPair +
- '}';
+ return "Node{"
+ + "outputStreams=" + outputStreams
+ + ", outputFields=" + outputFields
+ + ", groupingInfo=" + groupingInfo
+ + ", componentId='" + componentId + '\''
+ + ", parallelism=" + parallelism
+ + ", parentStreams=" + parentStreams
+ + ", windowed=" + windowed
+ + ", emitsPair=" + emitsPair
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/Pair.java b/storm-client/src/jvm/org/apache/storm/streams/Pair.java
index 348aeb1..ea065a4 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/Pair.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/Pair.java
@@ -22,23 +22,23 @@
*/
public final class Pair<T1, T2> implements Serializable {
/**
- * The first value
+ * The first value.
*/
- public final T1 _1;
+ public final T1 value1;
/**
- * The second value
+ * The second value.
*/
- public final T2 _2;
+ public final T2 value2;
/**
- * Constructs a new pair of values
+ * Constructs a new pair of values.
*
* @param first the first value
* @param second the second value
*/
private Pair(T1 first, T2 second) {
- _1 = first;
- _2 = second;
+ value1 = first;
+ value2 = second;
}
/**
@@ -60,7 +60,7 @@
* @return the first value
*/
public T1 getFirst() {
- return _1;
+ return value1;
}
/**
@@ -69,7 +69,7 @@
* @return the second value
*/
public T2 getSecond() {
- return _2;
+ return value2;
}
@Override
@@ -83,22 +83,22 @@
Pair<?, ?> pair = (Pair<?, ?>) o;
- if (_1 != null ? !_1.equals(pair._1) : pair._1 != null) {
+ if (value1 != null ? !value1.equals(pair.value1) : pair.value1 != null) {
return false;
}
- return _2 != null ? _2.equals(pair._2) : pair._2 == null;
+ return value2 != null ? value2.equals(pair.value2) : pair.value2 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + ", " + _2 + ')';
+ return "(" + value1 + ", " + value2 + ')';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/PairStream.java b/storm-client/src/jvm/org/apache/storm/streams/PairStream.java
index 57566c2..2d8e578 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/PairStream.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/PairStream.java
@@ -187,6 +187,28 @@
}
/**
+ * Join the values of this stream with the values having the same key from the other stream.
+ * <p>
+ * Note: The parallelism of this stream is carried forward to the joined stream.
+ * </p>
+ *
+ * @param otherStream the other stream
+ * @param valueJoiner the {@link ValueJoiner}
+ * @param <R> the type of the values resulting from the join
+ * @param <V1> the type of the values in the other stream
+ * @return the new stream
+ */
+ public <R, V1> PairStream<K, R> join(PairStream<K, V1> otherStream,
+ ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+ return partitionByKey()
+ .joinPartition(
+ otherStream.partitionByKey(),
+ valueJoiner,
+ JoinProcessor.JoinType.INNER,
+ JoinProcessor.JoinType.INNER);
+ }
+
+ /**
* Does a left outer join of the values of this stream with the values having the same key from the other stream.
* <p>
* Note: The parallelism of this stream is carried forward to the joined stream.
@@ -201,6 +223,28 @@
}
/**
+ * Does a left outer join of the values of this stream with the values having the same key from the other stream.
+ * <p>
+ * Note: The parallelism of this stream is carried forward to the joined stream.
+ * </p>
+ *
+ * @param otherStream the other stream
+ * @param valueJoiner the {@link ValueJoiner}
+ * @param <R> the type of the values resulting from the join
+ * @param <V1> the type of the values in the other stream
+ * @return the new stream
+ */
+ public <R, V1> PairStream<K, R> leftOuterJoin(PairStream<K, V1> otherStream,
+ ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+ return partitionByKey()
+ .joinPartition(
+ otherStream.partitionByKey(),
+ valueJoiner,
+ JoinProcessor.JoinType.OUTER,
+ JoinProcessor.JoinType.INNER);
+ }
+
+ /**
* Does a right outer join of the values of this stream with the values having the same key from the other stream.
* <p>
* Note: The parallelism of this stream is carried forward to the joined stream.
@@ -215,64 +259,6 @@
}
/**
- * Does a full outer join of the values of this stream with the values having the same key from the other stream.
- * <p>
- * Note: The parallelism of this stream is carried forward to the joined stream.
- * </p>
- *
- * @param otherStream the other stream
- * @param <V1> the type of the values in the other stream
- * @return the new stream
- */
- public <V1> PairStream<K, Pair<V, V1>> fullOuterJoin(PairStream<K, V1> otherStream) {
- return fullOuterJoin(otherStream, new PairValueJoiner<>());
- }
-
- /**
- * Join the values of this stream with the values having the same key from the other stream.
- * <p>
- * Note: The parallelism of this stream is carried forward to the joined stream.
- * </p>
- *
- * @param otherStream the other stream
- * @param valueJoiner the {@link ValueJoiner}
- * @param <R> the type of the values resulting from the join
- * @param <V1> the type of the values in the other stream
- * @return the new stream
- */
- public <R, V1> PairStream<K, R> join(PairStream<K, V1> otherStream,
- ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
- return partitionByKey()
- .joinPartition(
- otherStream.partitionByKey(),
- valueJoiner,
- JoinProcessor.JoinType.INNER,
- JoinProcessor.JoinType.INNER);
- }
-
- /**
- * Does a left outer join of the values of this stream with the values having the same key from the other stream.
- * <p>
- * Note: The parallelism of this stream is carried forward to the joined stream.
- * </p>
- *
- * @param otherStream the other stream
- * @param valueJoiner the {@link ValueJoiner}
- * @param <R> the type of the values resulting from the join
- * @param <V1> the type of the values in the other stream
- * @return the new stream
- */
- public <R, V1> PairStream<K, R> leftOuterJoin(PairStream<K, V1> otherStream,
- ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
- return partitionByKey()
- .joinPartition(
- otherStream.partitionByKey(),
- valueJoiner,
- JoinProcessor.JoinType.OUTER,
- JoinProcessor.JoinType.INNER);
- }
-
- /**
* Does a right outer join of the values of this stream with the values having the same key from the other stream.
* <p>
* Note: The parallelism of this stream is carried forward to the joined stream.
@@ -285,13 +271,27 @@
* @return the new stream
*/
public <R, V1> PairStream<K, R> rightOuterJoin(PairStream<K, V1> otherStream,
- ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+ ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
return partitionByKey()
- .joinPartition(
- otherStream.partitionByKey(),
- valueJoiner,
- JoinProcessor.JoinType.INNER,
- JoinProcessor.JoinType.OUTER);
+ .joinPartition(
+ otherStream.partitionByKey(),
+ valueJoiner,
+ JoinProcessor.JoinType.INNER,
+ JoinProcessor.JoinType.OUTER);
+ }
+
+ /**
+ * Does a full outer join of the values of this stream with the values having the same key from the other stream.
+ * <p>
+ * Note: The parallelism of this stream is carried forward to the joined stream.
+ * </p>
+ *
+ * @param otherStream the other stream
+ * @param <V1> the type of the values in the other stream
+ * @return the new stream
+ */
+ public <V1> PairStream<K, Pair<V, V1>> fullOuterJoin(PairStream<K, V1> otherStream) {
+ return fullOuterJoin(otherStream, new PairValueJoiner<>());
}
/**
@@ -307,13 +307,13 @@
* @return the new stream
*/
public <R, V1> PairStream<K, R> fullOuterJoin(PairStream<K, V1> otherStream,
- ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
+ ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner) {
return partitionByKey()
- .joinPartition(
- otherStream.partitionByKey(),
- valueJoiner,
- JoinProcessor.JoinType.OUTER,
- JoinProcessor.JoinType.OUTER);
+ .joinPartition(
+ otherStream.partitionByKey(),
+ valueJoiner,
+ JoinProcessor.JoinType.OUTER,
+ JoinProcessor.JoinType.OUTER);
}
/**
diff --git a/storm-client/src/jvm/org/apache/storm/streams/ProcessorBolt.java b/storm-client/src/jvm/org/apache/storm/streams/ProcessorBolt.java
index 8e078ef..559b40a 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/ProcessorBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/ProcessorBolt.java
@@ -23,7 +23,7 @@
import org.apache.storm.tuple.Tuple;
/**
- * Stream bolt that executes the different processors (except windowed and stateful operations)
+ * Stream bolt that executes the different processors (except windowed and stateful operations).
*/
class ProcessorBolt extends BaseRichBolt implements StreamBolt {
private final ProcessorBoltDelegate delegate;
diff --git a/storm-client/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java b/storm-client/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
index 901147e..f68746f 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java
@@ -289,8 +289,8 @@
for (String receivedStream : receivedStreams) {
Integer expected = streamToInputTaskCount.get(receivedStream);
if (expected == null) {
- throw new IllegalStateException("Punctuation received on unexpected stream '" + receivedStream +
- "' for which input task count is not set.");
+ throw new IllegalStateException("Punctuation received on unexpected stream '" + receivedStream
+ + "' for which input task count is not set.");
}
if (punctuationState.get(processorNode, receivedStream) < streamToInputTaskCount.get(receivedStream)) {
return false;
diff --git a/storm-client/src/jvm/org/apache/storm/streams/ProcessorNode.java b/storm-client/src/jvm/org/apache/storm/streams/ProcessorNode.java
index a2993d3..a92b5fe 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/ProcessorNode.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/ProcessorNode.java
@@ -67,11 +67,12 @@
@Override
public String toString() {
- return "ProcessorNode{" +
- "processor=" + processor +
- ", isBatch=" + isBatch +
- ", preservesKey=" + preservesKey +
- ", windowedParentStreams=" + windowedParentStreams +
- "} " + super.toString();
+ return "ProcessorNode{"
+ + "processor=" + processor
+ + ", isBatch=" + isBatch
+ + ", preservesKey=" + preservesKey
+ + ", windowedParentStreams=" + windowedParentStreams
+ + "} "
+ + super.toString();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/RefCountedTuple.java b/storm-client/src/jvm/org/apache/storm/streams/RefCountedTuple.java
index 96819e4..f887449 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/RefCountedTuple.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/RefCountedTuple.java
@@ -50,9 +50,9 @@
@Override
public String toString() {
- return "RefCountedTuple{" +
- "count=" + count +
- ", tuple=" + tuple +
- '}';
+ return "RefCountedTuple{"
+ + "count=" + count
+ + ", tuple=" + tuple
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/SpoutNode.java b/storm-client/src/jvm/org/apache/storm/streams/SpoutNode.java
index 4b86568..d59fe92 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/SpoutNode.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/SpoutNode.java
@@ -24,8 +24,8 @@
SpoutNode(IRichSpout spout) {
super(Utils.DEFAULT_STREAM_ID, getOutputFields(spout, Utils.DEFAULT_STREAM_ID));
if (outputFields.size() == 0) {
- throw new IllegalArgumentException("Spout " + spout + " does not declare any fields" +
- "for the stream '" + Utils.DEFAULT_STREAM_ID + "'");
+ throw new IllegalArgumentException("Spout " + spout + " does not declare any fields"
+ + "for the stream '" + Utils.DEFAULT_STREAM_ID + "'");
}
this.spout = spout;
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java b/storm-client/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
index a5e412f..1e7e922 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java
@@ -98,8 +98,8 @@
statefulProcessors.add((StatefulProcessor<K, V>) node.getProcessor());
if (node.getProcessor() instanceof UpdateStateByKeyProcessor) {
if (++updateStateByKeyCount > 1) {
- throw new IllegalArgumentException("Cannot have more than one updateStateByKey processor " +
- "in a StatefulProcessorBolt");
+ throw new IllegalArgumentException("Cannot have more than one updateStateByKey processor "
+ + "in a StatefulProcessorBolt");
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/Stream.java b/storm-client/src/jvm/org/apache/storm/streams/Stream.java
index c91b7a8..117b756 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/Stream.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/Stream.java
@@ -148,7 +148,7 @@
*
* // time duration based tumbling window
* stream.window(TumblingWindows.of(Duration.seconds(10));
- * </p>
+ * </pre>
*
* @see org.apache.storm.streams.windowing.SlidingWindows
* @see org.apache.storm.streams.windowing.TumblingWindows
@@ -381,18 +381,6 @@
return addNode(node, child);
}
- Node addProcessorNode(Processor<?> processor, Fields outputFields) {
- return addNode(makeProcessorNode(processor, outputFields));
- }
-
- Node addProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) {
- return addNode(makeProcessorNode(processor, outputFields, preservesKey));
- }
-
- String getStream() {
- return stream;
- }
-
private Node addNode(Node parent, Node child) {
return streamBuilder.addNode(parent, child);
}
@@ -405,6 +393,18 @@
return streamBuilder.addNode(parent, child, parentStreamId, parallelism);
}
+ Node addProcessorNode(Processor<?> processor, Fields outputFields) {
+ return addNode(makeProcessorNode(processor, outputFields));
+ }
+
+ Node addProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) {
+ return addNode(makeProcessorNode(processor, outputFields, preservesKey));
+ }
+
+ String getStream() {
+ return stream;
+ }
+
private ProcessorNode makeProcessorNode(Processor<?> processor, Fields outputFields) {
return makeProcessorNode(processor, outputFields, false);
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-client/src/jvm/org/apache/storm/streams/StreamBuilder.java
index d7ac83c..7827ada 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/StreamBuilder.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/StreamBuilder.java
@@ -51,7 +51,7 @@
import org.slf4j.LoggerFactory;
/**
- * A builder for constructing a {@link StormTopology} via storm streams api (DSL)
+ * A builder for constructing a {@link StormTopology} via storm streams api (DSL).
*/
@InterfaceStability.Unstable
public class StreamBuilder {
@@ -65,14 +65,14 @@
private String timestampFieldName = null;
/**
- * Creates a new {@link StreamBuilder}
+ * Creates a new {@link StreamBuilder}.
*/
public StreamBuilder() {
graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory());
}
/**
- * Creates a new {@link Stream} of tuples from the given {@link IRichSpout}
+ * Creates a new {@link Stream} of tuples from the given {@link IRichSpout}.
*
* @param spout the spout
* @return the new stream
@@ -196,18 +196,6 @@
return addNode(parent, child, parent.getOutputStreams().iterator().next(), parallelism);
}
- // insert child in-between parent and its current child nodes
- Node insert(Node parent, Node child) {
- Node newChild = addNode(parent, child);
- for (Edge edge : graph.outgoingEdgesOf(parent)) {
- Node oldChild = edge.getTarget();
- graph.removeEdge(parent, oldChild);
- oldChild.removeParentStreams(parent);
- addNode(newChild, oldChild);
- }
- return newChild;
- }
-
Node addNode(Node parent, Node child, String parentStreamId) {
return addNode(parent, child, parentStreamId, parent.getParallelism());
}
@@ -224,9 +212,13 @@
if (!(child instanceof PartitionNode)) {
if (child.getGroupingInfo() != null) {
if (!child.getGroupingInfo().equals(parent.getGroupingInfo())) {
- throw new IllegalStateException("Trying to assign grouping info for node" +
- " with current grouping info: " + child.getGroupingInfo() +
- " to: " + parent.getGroupingInfo() + " Node: " + child);
+ throw new IllegalStateException("Trying to assign grouping info for node"
+ + " with current grouping info: "
+ + child.getGroupingInfo()
+ + " to: "
+ + parent.getGroupingInfo()
+ + " Node: "
+ + child);
}
} else {
child.setGroupingInfo(parent.getGroupingInfo());
@@ -238,6 +230,18 @@
return child;
}
+ // insert child in-between parent and its current child nodes
+ Node insert(Node parent, Node child) {
+ Node newChild = addNode(parent, child);
+ for (Edge edge : graph.outgoingEdgesOf(parent)) {
+ Node oldChild = edge.getTarget();
+ graph.removeEdge(parent, oldChild);
+ oldChild.removeParentStreams(parent);
+ addNode(newChild, oldChild);
+ }
+ return newChild;
+ }
+
private PriorityQueue<Node> queue() {
// min-heap
return new PriorityQueue<>(new Comparator<Node>() {
@@ -248,16 +252,16 @@
* UpdateStateByKeyProcessor has a higher priority than StateQueryProcessor so that StateQueryProcessor
* can be mapped to the same StatefulBolt that UpdateStateByKeyProcessor is part of.
*/
- Map<Class<?>, Integer> p = new HashMap<>();
+ Map<Class<?>, Integer> map = new HashMap<>();
{
- p.put(SpoutNode.class, 0);
- p.put(UpdateStateByKeyProcessor.class, 1);
- p.put(ProcessorNode.class, 2);
- p.put(PartitionNode.class, 3);
- p.put(WindowNode.class, 4);
- p.put(StateQueryProcessor.class, 5);
- p.put(SinkNode.class, 6);
+ map.put(SpoutNode.class, 0);
+ map.put(UpdateStateByKeyProcessor.class, 1);
+ map.put(ProcessorNode.class, 2);
+ map.put(PartitionNode.class, 3);
+ map.put(WindowNode.class, 4);
+ map.put(StateQueryProcessor.class, 5);
+ map.put(SinkNode.class, 6);
}
@Override
@@ -270,12 +274,12 @@
// check if processor has specific priority first
if (node instanceof ProcessorNode) {
Processor processor = ((ProcessorNode) node).getProcessor();
- priority = p.get(processor.getClass());
+ priority = map.get(processor.getClass());
if (priority != null) {
return priority;
}
}
- priority = p.get(node.getClass());
+ priority = map.get(node.getClass());
if (priority != null) {
return priority;
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/StreamState.java b/storm-client/src/jvm/org/apache/storm/streams/StreamState.java
index c6de494..2f798a1 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/StreamState.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/StreamState.java
@@ -15,7 +15,7 @@
import java.io.Serializable;
/**
- * A wrapper for the stream state which can be used to query the state via {@link Stream#stateQuery(StreamState)}
+ * A wrapper for the stream state which can be used to query the state via {@link Stream#stateQuery(StreamState)}.
*
* @param <K> the key type
* @param <V> the value type
diff --git a/storm-client/src/jvm/org/apache/storm/streams/StreamUtil.java b/storm-client/src/jvm/org/apache/storm/streams/StreamUtil.java
index 2056b0a..d4b1029 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/StreamUtil.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/StreamUtil.java
@@ -12,13 +12,13 @@
package org.apache.storm.streams;
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
import java.util.ArrayList;
import java.util.List;
import org.apache.storm.shade.org.jgrapht.DirectedGraph;
import org.apache.storm.tuple.Fields;
-import static org.apache.storm.streams.WindowNode.PUNCTUATION;
-
public class StreamUtil {
@SuppressWarnings("unchecked")
public static <T> List<T> getParents(DirectedGraph<Node, Edge> graph, Node node) {
diff --git a/storm-client/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java b/storm-client/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
index 6891f31..eb5e09b 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java
@@ -12,6 +12,8 @@
package org.apache.storm.streams;
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
import java.util.Date;
import java.util.List;
import java.util.Map;
@@ -29,8 +31,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.streams.WindowNode.PUNCTUATION;
-
/**
* Stream bolt that executes windowing operations.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java b/storm-client/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java
index d95b618..344a842 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java
@@ -15,7 +15,7 @@
import org.apache.storm.streams.operations.CombinerAggregator;
/**
- * Computes the long sum of the input values
+ * Computes the long sum of the input values.
*/
public class LongSum implements CombinerAggregator<Number, Long, Long> {
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java b/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
index f21ebb8..50c0982 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java
@@ -30,8 +30,7 @@
private TupleValueMappers() {
}
- public static <T1, T2, T3>
- TupleValueMapper<Tuple3<T1, T2, T3>> of(int index1,
+ public static <T1, T2, T3> TupleValueMapper<Tuple3<T1, T2, T3>> of(int index1,
int index2,
int index3) {
return input -> new Tuple3<>(
@@ -40,8 +39,7 @@
(T3) input.getValue(index3));
}
- public static <T1, T2, T3, T4>
- TupleValueMapper<Tuple4<T1, T2, T3, T4>> of(int index1,
+ public static <T1, T2, T3, T4> TupleValueMapper<Tuple4<T1, T2, T3, T4>> of(int index1,
int index2,
int index3,
int index4) {
@@ -52,8 +50,7 @@
(T4) input.getValue(index4));
}
- public static <T1, T2, T3, T4, T5>
- TupleValueMapper<Tuple5<T1, T2, T3, T4, T5>> of(int index1,
+ public static <T1, T2, T3, T4, T5> TupleValueMapper<Tuple5<T1, T2, T3, T4, T5>> of(int index1,
int index2,
int index3,
int index4,
@@ -66,8 +63,7 @@
(T5) input.getValue(index5));
}
- public static <T1, T2, T3, T4, T5, T6>
- TupleValueMapper<Tuple6<T1, T2, T3, T4, T5, T6>> of(int index1,
+ public static <T1, T2, T3, T4, T5, T6> TupleValueMapper<Tuple6<T1, T2, T3, T4, T5, T6>> of(int index1,
int index2,
int index3,
int index4,
@@ -82,8 +78,7 @@
(T6) input.getValue(index6));
}
- public static <T1, T2, T3, T4, T5, T6, T7>
- TupleValueMapper<Tuple7<T1, T2, T3, T4, T5, T6, T7>> of(int index1,
+ public static <T1, T2, T3, T4, T5, T6, T7> TupleValueMapper<Tuple7<T1, T2, T3, T4, T5, T6, T7>> of(int index1,
int index2,
int index3,
int index4,
@@ -100,8 +95,7 @@
(T7) input.getValue(index7));
}
- public static <T1, T2, T3, T4, T5, T6, T7, T8>
- TupleValueMapper<Tuple8<T1, T2, T3, T4, T5, T6, T7, T8>> of(int index1,
+ public static <T1, T2, T3, T4, T5, T6, T7, T8> TupleValueMapper<Tuple8<T1, T2, T3, T4, T5, T6, T7, T8>> of(int index1,
int index2,
int index3,
int index4,
@@ -120,8 +114,7 @@
(T8) input.getValue(index8));
}
- public static <T1, T2, T3, T4, T5, T6, T7, T8, T9>
- TupleValueMapper<Tuple9<T1, T2, T3, T4, T5, T6, T7, T8, T9>> of(int index1,
+ public static <T1, T2, T3, T4, T5, T6, T7, T8, T9> TupleValueMapper<Tuple9<T1, T2, T3, T4, T5, T6, T7, T8, T9>> of(int index1,
int index2,
int index3,
int index4,
@@ -142,8 +135,9 @@
(T9) input.getValue(index9));
}
+ @SuppressWarnings("checkstyle:MethodTypeParameterName")
public static <T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>
- TupleValueMapper<Tuple10<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> of(int index1,
+ TupleValueMapper<Tuple10<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> of(int index1,
int index2,
int index3,
int index4,
diff --git a/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java b/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
index 8536bd6..40838df 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java
@@ -22,7 +22,7 @@
private final int[] indices;
/**
- * Constructs a new {@link ValuesMapper} that extracts value from a {@link Tuple} at specified indices
+ * Constructs a new {@link ValuesMapper} that extracts value from a {@link Tuple} at specified indices.
*
* @param indices the indices
*/
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
index 2c0457b..7830729 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java
@@ -62,10 +62,10 @@
@Override
public String toString() {
- return "AggregateByKeyProcessor{" +
- "aggregator=" + aggregator +
- ", emitAggregate=" + emitAggregate +
- ", state=" + state +
- "}";
+ return "AggregateByKeyProcessor{"
+ + "aggregator=" + aggregator
+ + ", emitAggregate=" + emitAggregate
+ + ", state=" + state
+ + "}";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
index 7c251cd..952d7ce 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java
@@ -55,10 +55,10 @@
@Override
public String toString() {
- return "AggregateProcessor{" +
- "aggregator=" + aggregator +
- ", emitAggregate=" + emitAggregate +
- ", state=" + state +
- "}";
+ return "AggregateProcessor{"
+ + "aggregator=" + aggregator
+ + ", emitAggregate=" + emitAggregate
+ + ", state=" + state
+ + "}";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
index ea44949..06486ac 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java
@@ -12,16 +12,14 @@
package org.apache.storm.streams.processors;
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
import java.util.HashSet;
import java.util.Set;
import java.util.function.Supplier;
-import static org.apache.storm.streams.WindowNode.PUNCTUATION;
-
/**
* Base implementation of the {@link Processor} interface that provides convenience methods {@link #execute(Object)} and {@link #finish()}.
- *
- * @param <T>
*/
abstract class BaseProcessor<T> implements Processor<T> {
private final Set<String> punctuationState = new HashSet<>();
@@ -44,6 +42,16 @@
}
/**
+ * Execute some operation on the input value. Sub classes can override this when then don't care about the source stream from where the
+ * input is received.
+ *
+ * @param input the input
+ */
+ protected void execute(T input) {
+ // NOOP
+ }
+
+ /**
* {@inheritDoc}
*/
@Override
@@ -56,16 +64,6 @@
}
/**
- * Execute some operation on the input value. Sub classes can override this when then don't care about the source stream from where the
- * input is received.
- *
- * @param input the input
- */
- protected void execute(T input) {
- // NOOP
- }
-
- /**
* This is triggered to signal the end of the current batch of values. Sub classes can override this to emit the result of a batch of
* values, for e.g. to emit the result of an aggregate or join operation on a batch of values. If a processor does per-value operation
* like filter, map etc, they can choose to ignore this.
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java b/storm-client/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
index e4053e3..de39004 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java
@@ -19,7 +19,7 @@
import org.apache.storm.streams.ProcessorNode;
/**
- * A composite context that holds a chain of {@link ProcessorContext}
+ * A composite context that holds a chain of {@link ProcessorContext}.
*/
public class ChainedProcessorContext implements ProcessorContext {
private final ProcessorNode processorNode;
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessor.java
index 04eeea7..af638cb 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/CoGroupByKeyProcessor.java
@@ -18,7 +18,7 @@
import org.apache.storm.streams.Pair;
/**
- * co-group by key implementation
+ * co-group by key implementation.
*/
public class CoGroupByKeyProcessor<K, V1, V2> extends BaseProcessor<Pair<K, ?>> implements BatchProcessor {
private final String firstStream;
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java b/storm-client/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
index 4dfcaa3..de6fc4f 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java
@@ -12,6 +12,8 @@
package org.apache.storm.streams.processors;
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@@ -28,8 +30,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.streams.WindowNode.PUNCTUATION;
-
/**
* A context that emits the results to downstream processors which are in another bolt.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java b/storm-client/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
index c9ef345..450d326 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java
@@ -12,14 +12,14 @@
package org.apache.storm.streams.processors;
+import static org.apache.storm.streams.WindowNode.PUNCTUATION;
+
import java.util.Set;
import org.apache.storm.shade.com.google.common.collect.Multimap;
import org.apache.storm.streams.ProcessorNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.streams.WindowNode.PUNCTUATION;
-
/**
* A context that emits the results to downstream processors which are in the same bolt.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
index c6727de..6295c26 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java
@@ -90,11 +90,11 @@
private void joinAndForward(List<Pair<K, V1>> leftRows, List<Pair<K, V2>> rightRows) {
if (leftRows.size() < rightRows.size()) {
for (Tuple3<K, V1, V2> res : join(getJoinTable(leftRows), rightRows, leftType, rightType)) {
- context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3)));
+ context.forward(Pair.of(res.value1, valueJoiner.apply(res.value2, res.value3)));
}
} else {
for (Tuple3<K, V2, V1> res : join(getJoinTable(rightRows), leftRows, rightType, leftType)) {
- context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2)));
+ context.forward(Pair.of(res.value1, valueJoiner.apply(res.value3, res.value2)));
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java b/storm-client/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
index d1cfd1e..68ae103 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java
@@ -22,7 +22,7 @@
*/
public interface StatefulProcessor<K, V> {
/**
- * Initialize the state of the processor with the given {@link KeyValueState}
+ * Initialize the state of the processor with the given {@link KeyValueState}.
*
* @param keyValueState the key-value state
*/
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple10.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple10.java
index d0c598c..38d08f2 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple10.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple10.java
@@ -26,43 +26,44 @@
* @param <T9> the type of the ninth element
* @param <T10> the type of the tenth element
*/
+@SuppressWarnings("checkstyle:ClassTypeParameterName")
public class Tuple10<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
- public final T6 _6;
- public final T7 _7;
- public final T8 _8;
- public final T9 _9;
- public final T10 _10;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
+ public final T6 value6;
+ public final T7 value7;
+ public final T8 value8;
+ public final T9 value9;
+ public final T10 value10;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
- * @param _6 the sixth element
- * @param _7 the seventh element
- * @param _8 the eighth element
- * @param _9 the ninth element
- * @param _10 the tenth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
+ * @param value6 the sixth element
+ * @param value7 the seventh element
+ * @param value8 the eighth element
+ * @param value9 the ninth element
+ * @param value10 the tenth element
*/
- public Tuple10(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9, T10 _10) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
- this._6 = _6;
- this._7 = _7;
- this._8 = _8;
- this._9 = _9;
- this._10 = _10;
+ public Tuple10(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8, T9 value9, T10 value10) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
+ this.value6 = value6;
+ this.value7 = value7;
+ this.value8 = value8;
+ this.value9 = value9;
+ this.value10 = value10;
}
@Override
@@ -76,54 +77,64 @@
Tuple10<?, ?, ?, ?, ?, ?, ?, ?, ?, ?> tuple10 = (Tuple10<?, ?, ?, ?, ?, ?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple10._1) : tuple10._1 != null) {
+ if (value1 != null ? !value1.equals(tuple10.value1) : tuple10.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple10._2) : tuple10._2 != null) {
+ if (value2 != null ? !value2.equals(tuple10.value2) : tuple10.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple10._3) : tuple10._3 != null) {
+ if (value3 != null ? !value3.equals(tuple10.value3) : tuple10.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple10._4) : tuple10._4 != null) {
+ if (value4 != null ? !value4.equals(tuple10.value4) : tuple10.value4 != null) {
return false;
}
- if (_5 != null ? !_5.equals(tuple10._5) : tuple10._5 != null) {
+ if (value5 != null ? !value5.equals(tuple10.value5) : tuple10.value5 != null) {
return false;
}
- if (_6 != null ? !_6.equals(tuple10._6) : tuple10._6 != null) {
+ if (value6 != null ? !value6.equals(tuple10.value6) : tuple10.value6 != null) {
return false;
}
- if (_7 != null ? !_7.equals(tuple10._7) : tuple10._7 != null) {
+ if (value7 != null ? !value7.equals(tuple10.value7) : tuple10.value7 != null) {
return false;
}
- if (_8 != null ? !_8.equals(tuple10._8) : tuple10._8 != null) {
+ if (value8 != null ? !value8.equals(tuple10.value8) : tuple10.value8 != null) {
return false;
}
- if (_9 != null ? !_9.equals(tuple10._9) : tuple10._9 != null) {
+ if (value9 != null ? !value9.equals(tuple10.value9) : tuple10.value9 != null) {
return false;
}
- return _10 != null ? _10.equals(tuple10._10) : tuple10._10 == null;
+ return value10 != null ? value10.equals(tuple10.value10) : tuple10.value10 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
- result = 31 * result + (_6 != null ? _6.hashCode() : 0);
- result = 31 * result + (_7 != null ? _7.hashCode() : 0);
- result = 31 * result + (_8 != null ? _8.hashCode() : 0);
- result = 31 * result + (_9 != null ? _9.hashCode() : 0);
- result = 31 * result + (_10 != null ? _10.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
+ result = 31 * result + (value6 != null ? value6.hashCode() : 0);
+ result = 31 * result + (value7 != null ? value7.hashCode() : 0);
+ result = 31 * result + (value8 != null ? value8.hashCode() : 0);
+ result = 31 * result + (value9 != null ? value9.hashCode() : 0);
+ result = 31 * result + (value10 != null ? value10.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + "," + _10 + ")";
+ return "(" + value1
+ + "," + value2
+ + "," + value3
+ + "," + value4
+ + "," + value5
+ + "," + value6
+ + "," + value7
+ + "," + value8
+ + "," + value9
+ + "," + value10
+ + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple3.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple3.java
index 1806f80..ebf1477 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple3.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple3.java
@@ -20,21 +20,21 @@
* @param <T3> the type of the third element
*/
public final class Tuple3<T1, T2, T3> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
*/
- public Tuple3(T1 _1, T2 _2, T3 _3) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
+ public Tuple3(T1 value1, T2 value2, T3 value3) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
}
@Override
@@ -48,26 +48,26 @@
Tuple3<?, ?, ?> tuple3 = (Tuple3<?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple3._1) : tuple3._1 != null) {
+ if (value1 != null ? !value1.equals(tuple3.value1) : tuple3.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple3._2) : tuple3._2 != null) {
+ if (value2 != null ? !value2.equals(tuple3.value2) : tuple3.value2 != null) {
return false;
}
- return _3 != null ? _3.equals(tuple3._3) : tuple3._3 == null;
+ return value3 != null ? value3.equals(tuple3.value3) : tuple3.value3 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple4.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
index df01f5f..e457715 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple4.java
@@ -21,24 +21,24 @@
* @param <T4> the type of the fourth element
*/
public final class Tuple4<T1, T2, T3, T4> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
*/
- public Tuple4(T1 _1, T2 _2, T3 _3, T4 _4) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
+ public Tuple4(T1 value1, T2 value2, T3 value3, T4 value4) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
}
@Override
@@ -52,30 +52,30 @@
Tuple4<?, ?, ?, ?> tuple4 = (Tuple4<?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple4._1) : tuple4._1 != null) {
+ if (value1 != null ? !value1.equals(tuple4.value1) : tuple4.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple4._2) : tuple4._2 != null) {
+ if (value2 != null ? !value2.equals(tuple4.value2) : tuple4.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple4._3) : tuple4._3 != null) {
+ if (value3 != null ? !value3.equals(tuple4.value3) : tuple4.value3 != null) {
return false;
}
- return _4 != null ? _4.equals(tuple4._4) : tuple4._4 == null;
+ return value4 != null ? value4.equals(tuple4.value4) : tuple4.value4 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + "," + value4 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple5.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
index 7c9cb84..21d5271 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple5.java
@@ -22,27 +22,27 @@
* @param <T5> the type of the fifth element
*/
public class Tuple5<T1, T2, T3, T4, T5> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
*/
- public Tuple5(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
+ public Tuple5(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
}
@Override
@@ -56,34 +56,34 @@
Tuple5<?, ?, ?, ?, ?> tuple5 = (Tuple5<?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple5._1) : tuple5._1 != null) {
+ if (value1 != null ? !value1.equals(tuple5.value1) : tuple5.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple5._2) : tuple5._2 != null) {
+ if (value2 != null ? !value2.equals(tuple5.value2) : tuple5.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple5._3) : tuple5._3 != null) {
+ if (value3 != null ? !value3.equals(tuple5.value3) : tuple5.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple5._4) : tuple5._4 != null) {
+ if (value4 != null ? !value4.equals(tuple5.value4) : tuple5.value4 != null) {
return false;
}
- return _5 != null ? _5.equals(tuple5._5) : tuple5._5 == null;
+ return value5 != null ? value5.equals(tuple5.value5) : tuple5.value5 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + "," + value4 + "," + value5 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple6.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
index dfd209c..6891f43 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple6.java
@@ -24,30 +24,30 @@
* @param <T6> the type of the sixth element
*/
public class Tuple6<T1, T2, T3, T4, T5, T6> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
- public final T6 _6;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
+ public final T6 value6;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
- * @param _6 the sixth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
+ * @param value6 the sixth element
*/
- public Tuple6(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
- this._6 = _6;
+ public Tuple6(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
+ this.value6 = value6;
}
@Override
@@ -61,38 +61,38 @@
Tuple6<?, ?, ?, ?, ?, ?> tuple6 = (Tuple6<?, ?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple6._1) : tuple6._1 != null) {
+ if (value1 != null ? !value1.equals(tuple6.value1) : tuple6.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple6._2) : tuple6._2 != null) {
+ if (value2 != null ? !value2.equals(tuple6.value2) : tuple6.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple6._3) : tuple6._3 != null) {
+ if (value3 != null ? !value3.equals(tuple6.value3) : tuple6.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple6._4) : tuple6._4 != null) {
+ if (value4 != null ? !value4.equals(tuple6.value4) : tuple6.value4 != null) {
return false;
}
- if (_5 != null ? !_5.equals(tuple6._5) : tuple6._5 != null) {
+ if (value5 != null ? !value5.equals(tuple6.value5) : tuple6.value5 != null) {
return false;
}
- return _6 != null ? _6.equals(tuple6._6) : tuple6._6 == null;
+ return value6 != null ? value6.equals(tuple6.value6) : tuple6.value6 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
- result = 31 * result + (_6 != null ? _6.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
+ result = 31 * result + (value6 != null ? value6.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + "," + value4 + "," + value5 + "," + value6 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple7.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
index 34301cb..c2aab42 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple7.java
@@ -24,33 +24,33 @@
* @param <T7> the type of the seventh element
*/
public class Tuple7<T1, T2, T3, T4, T5, T6, T7> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
- public final T6 _6;
- public final T7 _7;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
+ public final T6 value6;
+ public final T7 value7;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
- * @param _6 the sixth element
- * @param _7 the seventh element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
+ * @param value6 the sixth element
+ * @param value7 the seventh element
*/
- public Tuple7(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
- this._6 = _6;
- this._7 = _7;
+ public Tuple7(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
+ this.value6 = value6;
+ this.value7 = value7;
}
@Override
@@ -64,42 +64,42 @@
Tuple7<?, ?, ?, ?, ?, ?, ?> tuple7 = (Tuple7<?, ?, ?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple7._1) : tuple7._1 != null) {
+ if (value1 != null ? !value1.equals(tuple7.value1) : tuple7.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple7._2) : tuple7._2 != null) {
+ if (value2 != null ? !value2.equals(tuple7.value2) : tuple7.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple7._3) : tuple7._3 != null) {
+ if (value3 != null ? !value3.equals(tuple7.value3) : tuple7.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple7._4) : tuple7._4 != null) {
+ if (value4 != null ? !value4.equals(tuple7.value4) : tuple7.value4 != null) {
return false;
}
- if (_5 != null ? !_5.equals(tuple7._5) : tuple7._5 != null) {
+ if (value5 != null ? !value5.equals(tuple7.value5) : tuple7.value5 != null) {
return false;
}
- if (_6 != null ? !_6.equals(tuple7._6) : tuple7._6 != null) {
+ if (value6 != null ? !value6.equals(tuple7.value6) : tuple7.value6 != null) {
return false;
}
- return _7 != null ? _7.equals(tuple7._7) : tuple7._7 == null;
+ return value7 != null ? value7.equals(tuple7.value7) : tuple7.value7 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
- result = 31 * result + (_6 != null ? _6.hashCode() : 0);
- result = 31 * result + (_7 != null ? _7.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
+ result = 31 * result + (value6 != null ? value6.hashCode() : 0);
+ result = 31 * result + (value7 != null ? value7.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + "," + value4 + "," + value5 + "," + value6 + "," + value7 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple8.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
index 54a75be..11a85f8 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple8.java
@@ -25,36 +25,36 @@
* @param <T8> the type of the eighth element
*/
public class Tuple8<T1, T2, T3, T4, T5, T6, T7, T8> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
- public final T6 _6;
- public final T7 _7;
- public final T8 _8;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
+ public final T6 value6;
+ public final T7 value7;
+ public final T8 value8;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
- * @param _6 the sixth element
- * @param _7 the seventh element
- * @param _8 the eighth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
+ * @param value6 the sixth element
+ * @param value7 the seventh element
+ * @param value8 the eighth element
*/
- public Tuple8(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
- this._6 = _6;
- this._7 = _7;
- this._8 = _8;
+ public Tuple8(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
+ this.value6 = value6;
+ this.value7 = value7;
+ this.value8 = value8;
}
@Override
@@ -68,46 +68,46 @@
Tuple8<?, ?, ?, ?, ?, ?, ?, ?> tuple8 = (Tuple8<?, ?, ?, ?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple8._1) : tuple8._1 != null) {
+ if (value1 != null ? !value1.equals(tuple8.value1) : tuple8.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple8._2) : tuple8._2 != null) {
+ if (value2 != null ? !value2.equals(tuple8.value2) : tuple8.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple8._3) : tuple8._3 != null) {
+ if (value3 != null ? !value3.equals(tuple8.value3) : tuple8.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple8._4) : tuple8._4 != null) {
+ if (value4 != null ? !value4.equals(tuple8.value4) : tuple8.value4 != null) {
return false;
}
- if (_5 != null ? !_5.equals(tuple8._5) : tuple8._5 != null) {
+ if (value5 != null ? !value5.equals(tuple8.value5) : tuple8.value5 != null) {
return false;
}
- if (_6 != null ? !_6.equals(tuple8._6) : tuple8._6 != null) {
+ if (value6 != null ? !value6.equals(tuple8.value6) : tuple8.value6 != null) {
return false;
}
- if (_7 != null ? !_7.equals(tuple8._7) : tuple8._7 != null) {
+ if (value7 != null ? !value7.equals(tuple8.value7) : tuple8.value7 != null) {
return false;
}
- return _8 != null ? _8.equals(tuple8._8) : tuple8._8 == null;
+ return value8 != null ? value8.equals(tuple8.value8) : tuple8.value8 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
- result = 31 * result + (_6 != null ? _6.hashCode() : 0);
- result = 31 * result + (_7 != null ? _7.hashCode() : 0);
- result = 31 * result + (_8 != null ? _8.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
+ result = 31 * result + (value6 != null ? value6.hashCode() : 0);
+ result = 31 * result + (value7 != null ? value7.hashCode() : 0);
+ result = 31 * result + (value8 != null ? value8.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + ")";
+ return "(" + value1 + "," + value2 + "," + value3 + "," + value4 + "," + value5 + "," + value6 + "," + value7 + "," + value8 + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple9.java b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
index a6f4e17..9f64b18 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/tuple/Tuple9.java
@@ -26,39 +26,39 @@
* @param <T9> the type of the ninth element
*/
public class Tuple9<T1, T2, T3, T4, T5, T6, T7, T8, T9> {
- public final T1 _1;
- public final T2 _2;
- public final T3 _3;
- public final T4 _4;
- public final T5 _5;
- public final T6 _6;
- public final T7 _7;
- public final T8 _8;
- public final T9 _9;
+ public final T1 value1;
+ public final T2 value2;
+ public final T3 value3;
+ public final T4 value4;
+ public final T5 value5;
+ public final T6 value6;
+ public final T7 value7;
+ public final T8 value8;
+ public final T9 value9;
/**
* Constructs a new tuple.
*
- * @param _1 the first element
- * @param _2 the second element
- * @param _3 the third element
- * @param _4 the fourth element
- * @param _5 the fifth element
- * @param _6 the sixth element
- * @param _7 the seventh element
- * @param _8 the eighth element
- * @param _9 the ninth element
+ * @param value1 the first element
+ * @param value2 the second element
+ * @param value3 the third element
+ * @param value4 the fourth element
+ * @param value5 the fifth element
+ * @param value6 the sixth element
+ * @param value7 the seventh element
+ * @param value8 the eighth element
+ * @param value9 the ninth element
*/
- public Tuple9(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9) {
- this._1 = _1;
- this._2 = _2;
- this._3 = _3;
- this._4 = _4;
- this._5 = _5;
- this._6 = _6;
- this._7 = _7;
- this._8 = _8;
- this._9 = _9;
+ public Tuple9(T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8, T9 value9) {
+ this.value1 = value1;
+ this.value2 = value2;
+ this.value3 = value3;
+ this.value4 = value4;
+ this.value5 = value5;
+ this.value6 = value6;
+ this.value7 = value7;
+ this.value8 = value8;
+ this.value9 = value9;
}
@Override
@@ -72,50 +72,58 @@
Tuple9<?, ?, ?, ?, ?, ?, ?, ?, ?> tuple9 = (Tuple9<?, ?, ?, ?, ?, ?, ?, ?, ?>) o;
- if (_1 != null ? !_1.equals(tuple9._1) : tuple9._1 != null) {
+ if (value1 != null ? !value1.equals(tuple9.value1) : tuple9.value1 != null) {
return false;
}
- if (_2 != null ? !_2.equals(tuple9._2) : tuple9._2 != null) {
+ if (value2 != null ? !value2.equals(tuple9.value2) : tuple9.value2 != null) {
return false;
}
- if (_3 != null ? !_3.equals(tuple9._3) : tuple9._3 != null) {
+ if (value3 != null ? !value3.equals(tuple9.value3) : tuple9.value3 != null) {
return false;
}
- if (_4 != null ? !_4.equals(tuple9._4) : tuple9._4 != null) {
+ if (value4 != null ? !value4.equals(tuple9.value4) : tuple9.value4 != null) {
return false;
}
- if (_5 != null ? !_5.equals(tuple9._5) : tuple9._5 != null) {
+ if (value5 != null ? !value5.equals(tuple9.value5) : tuple9.value5 != null) {
return false;
}
- if (_6 != null ? !_6.equals(tuple9._6) : tuple9._6 != null) {
+ if (value6 != null ? !value6.equals(tuple9.value6) : tuple9.value6 != null) {
return false;
}
- if (_7 != null ? !_7.equals(tuple9._7) : tuple9._7 != null) {
+ if (value7 != null ? !value7.equals(tuple9.value7) : tuple9.value7 != null) {
return false;
}
- if (_8 != null ? !_8.equals(tuple9._8) : tuple9._8 != null) {
+ if (value8 != null ? !value8.equals(tuple9.value8) : tuple9.value8 != null) {
return false;
}
- return _9 != null ? _9.equals(tuple9._9) : tuple9._9 == null;
-
+ return value9 != null ? value9.equals(tuple9.value9) : tuple9.value9 == null;
}
@Override
public int hashCode() {
- int result = _1 != null ? _1.hashCode() : 0;
- result = 31 * result + (_2 != null ? _2.hashCode() : 0);
- result = 31 * result + (_3 != null ? _3.hashCode() : 0);
- result = 31 * result + (_4 != null ? _4.hashCode() : 0);
- result = 31 * result + (_5 != null ? _5.hashCode() : 0);
- result = 31 * result + (_6 != null ? _6.hashCode() : 0);
- result = 31 * result + (_7 != null ? _7.hashCode() : 0);
- result = 31 * result + (_8 != null ? _8.hashCode() : 0);
- result = 31 * result + (_9 != null ? _9.hashCode() : 0);
+ int result = value1 != null ? value1.hashCode() : 0;
+ result = 31 * result + (value2 != null ? value2.hashCode() : 0);
+ result = 31 * result + (value3 != null ? value3.hashCode() : 0);
+ result = 31 * result + (value4 != null ? value4.hashCode() : 0);
+ result = 31 * result + (value5 != null ? value5.hashCode() : 0);
+ result = 31 * result + (value6 != null ? value6.hashCode() : 0);
+ result = 31 * result + (value7 != null ? value7.hashCode() : 0);
+ result = 31 * result + (value8 != null ? value8.hashCode() : 0);
+ result = 31 * result + (value9 != null ? value9.hashCode() : 0);
return result;
}
@Override
public String toString() {
- return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + ")";
+ return "(" + value1
+ + "," + value2
+ + "," + value3
+ + "," + value4
+ + "," + value5
+ + "," + value6
+ + "," + value7
+ + "," + value8
+ + "," + value9
+ + ")";
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java b/storm-client/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
index 8b0863a..faec6bf 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java
@@ -12,11 +12,11 @@
package org.apache.storm.streams.windowing;
-import org.apache.storm.topology.base.BaseWindowedBolt;
-
import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+
/**
* A tumbling window specification. The window tumbles after the specified window length.
*
diff --git a/storm-client/src/jvm/org/apache/storm/streams/windowing/Window.java b/storm-client/src/jvm/org/apache/storm/streams/windowing/Window.java
index 8da2158..986426f 100644
--- a/storm-client/src/jvm/org/apache/storm/streams/windowing/Window.java
+++ b/storm-client/src/jvm/org/apache/storm/streams/windowing/Window.java
@@ -12,10 +12,10 @@
package org.apache.storm.streams.windowing;
-import java.io.Serializable;
-
import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+import java.io.Serializable;
+
/**
* The window specification within {@link org.apache.storm.streams.Stream}.
*
diff --git a/storm-client/src/jvm/org/apache/storm/task/GeneralTopologyContext.java b/storm-client/src/jvm/org/apache/storm/task/GeneralTopologyContext.java
index 0dd2a2f..ac4c850 100644
--- a/storm-client/src/jvm/org/apache/storm/task/GeneralTopologyContext.java
+++ b/storm-client/src/jvm/org/apache/storm/task/GeneralTopologyContext.java
@@ -38,25 +38,25 @@
import org.apache.storm.utils.ThriftTopologyUtils;
public class GeneralTopologyContext implements JSONAware {
- protected Map<String, Object> _topoConf;
- protected boolean _doSanityCheck;
- private StormTopology _topology;
- private Map<Integer, String> _taskToComponent;
- private Map<String, List<Integer>> _componentToTasks;
- private Map<String, Map<String, Fields>> _componentToStreamToFields;
- private String _stormId;
+ protected Map<String, Object> topoConf;
+ protected boolean doSanityCheck;
+ private StormTopology topology;
+ private Map<Integer, String> taskToComponent;
+ private Map<String, List<Integer>> componentToTasks;
+ private Map<String, Map<String, Fields>> componentToStreamToFields;
+ private String stormId;
// pass in componentToSortedTasks for the case of running tons of tasks in single executor
public GeneralTopologyContext(StormTopology topology, Map<String, Object> topoConf,
Map<Integer, String> taskToComponent, Map<String, List<Integer>> componentToSortedTasks,
Map<String, Map<String, Fields>> componentToStreamToFields, String stormId) {
- _topology = topology;
- _topoConf = topoConf;
- _taskToComponent = taskToComponent;
- _stormId = stormId;
- _componentToTasks = componentToSortedTasks;
- _componentToStreamToFields = componentToStreamToFields;
- _doSanityCheck = ConfigUtils.isLocalMode(_topoConf);
+ this.topology = topology;
+ this.topoConf = topoConf;
+ this.taskToComponent = taskToComponent;
+ this.stormId = stormId;
+ componentToTasks = componentToSortedTasks;
+ this.componentToStreamToFields = componentToStreamToFields;
+ doSanityCheck = ConfigUtils.isLocalMode(this.topoConf);
}
/**
@@ -65,7 +65,7 @@
* @return the storm id
*/
public String getStormId() {
- return _stormId;
+ return stormId;
}
/**
@@ -74,7 +74,7 @@
* @return the Thrift definition representing the topology
*/
public StormTopology getRawTopology() {
- return _topology;
+ return topology;
}
/**
@@ -88,7 +88,7 @@
if (taskId == Constants.SYSTEM_TASK_ID) {
return Constants.SYSTEM_COMPONENT_ID;
} else {
- return _taskToComponent.get(taskId);
+ return taskToComponent.get(taskId);
}
}
@@ -103,7 +103,7 @@
* Gets the task ids allocated for the given component id. The task ids are always returned in ascending order.
*/
public List<Integer> getComponentTasks(String componentId) {
- List<Integer> ret = _componentToTasks.get(componentId);
+ List<Integer> ret = componentToTasks.get(componentId);
if (ret == null) {
return new ArrayList<>();
} else {
@@ -115,7 +115,7 @@
* Gets the declared output fields for the specified component/stream.
*/
public Fields getComponentOutputFields(String componentId, String streamId) {
- Fields ret = _componentToStreamToFields.get(componentId).get(streamId);
+ Fields ret = componentToStreamToFields.get(componentId).get(streamId);
if (ret == null) {
throw new IllegalArgumentException("No output fields defined for component:stream " + componentId + ":" + streamId);
}
@@ -165,7 +165,7 @@
@Override
public String toJSONString() {
Map<String, Object> obj = new HashMap<>();
- obj.put("task->component", _taskToComponent);
+ obj.put("task->component", taskToComponent);
// TODO: jsonify StormTopology
// at the minimum should send source info
return JSONValue.toJSONString(obj);
@@ -175,11 +175,11 @@
* Gets a map from task id to component id.
*/
public Map<Integer, String> getTaskToComponent() {
- return _taskToComponent;
+ return taskToComponent;
}
/**
- * Gets a list of all component ids in this topology
+ * Gets a list of all component ids in this topology.
*/
public Set<String> getComponentIds() {
return ThriftTopologyUtils.getComponentIds(getRawTopology());
@@ -190,7 +190,7 @@
}
public int maxTopologyMessageTimeout() {
- Integer max = ObjectReader.getInt(_topoConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS));
+ Integer max = ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS));
for (String spout : getRawTopology().get_spouts().keySet()) {
ComponentCommon common = getComponentCommon(spout);
String jsonConf = common.get_json_conf();
@@ -208,10 +208,10 @@
}
public Map<String, Object> getConf() {
- return _topoConf;
+ return topoConf;
}
public boolean doSanityCheck() {
- return _doSanityCheck;
+ return doSanityCheck;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/task/IBolt.java b/storm-client/src/jvm/org/apache/storm/task/IBolt.java
index 6e0d033..ad47ea5 100644
--- a/storm-client/src/jvm/org/apache/storm/task/IBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/task/IBolt.java
@@ -21,23 +21,24 @@
* An IBolt represents a component that takes tuples as input and produces tuples as output. An IBolt can do everything from filtering to
* joining to functions to aggregations. It does not have to process a tuple immediately and may hold onto tuples to process later.
*
- * A bolt's lifecycle is as follows:
+ * <p>A bolt's lifecycle is as follows:
*
- * IBolt object created on client machine. The IBolt is serialized into the topology (using Java serialization) and submitted to the master
- * machine of the cluster (Nimbus). Nimbus then launches workers which deserialize the object, call prepare on it, and then start processing
- * tuples.
+ * <p>IBolt object created on client machine. The IBolt is serialized into the topology (using Java serialization) and
+ * submitted to the master machine of the cluster (Nimbus). Nimbus then launches workers which deserialize the object,
+ * call prepare on it, and then start processing tuples.
*
- * If you want to parameterize an IBolt, you should set the parameters through its constructor and save the parameterization state as
+ * <p>If you want to parameterize an IBolt, you should set the parameters through its constructor and save the parameterization state as
* instance variables (which will then get serialized and shipped to every task executing this bolt across the cluster).
*
- * When defining bolts in Java, you should use the IRichBolt interface which adds necessary methods for using the Java TopologyBuilder API.
+ * <p>When defining bolts in Java, you should use the IRichBolt interface which adds necessary methods for using the
+ * Java TopologyBuilder API.
*/
public interface IBolt extends Serializable {
/**
* Called when a task for this component is initialized within a worker on the cluster. It provides the bolt with the environment in
* which the bolt executes.
*
- * This includes the:
+ * <p>This includes the:
*
* @param topoConf The Storm configuration for this bolt. This is the configuration provided to the topology merged in with cluster
* configuration on this machine.
@@ -53,11 +54,11 @@
* of the Tuple can be accessed using Tuple#getValue. The IBolt does not have to process the Tuple immediately. It is perfectly fine to
* hang onto a tuple and process it later (for instance, to do an aggregation or join).
*
- * Tuples should be emitted using the OutputCollector provided through the prepare method. It is required that all input tuples are
+ * <p>Tuples should be emitted using the OutputCollector provided through the prepare method. It is required that all input tuples are
* acked or failed at some point using the OutputCollector. Otherwise, Storm will be unable to determine when tuples coming off the
* spouts have been completed.
*
- * For the common case of acking an input tuple at the end of the execute method, see IBasicBolt which automates this.
+ * <p>For the common case of acking an input tuple at the end of the execute method, see IBasicBolt which automates this.
*
* @param input The input tuple to be processed.
*/
@@ -68,7 +69,7 @@
* The {@link Config#SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS} setting controls how long orderly shutdown is allowed to take.
* There is no guarantee that cleanup will be called if shutdown is not orderly, or if the shutdown exceeds the time limit.
*
- * The one context where cleanup is guaranteed to be called is when a topology is killed when running Storm in local mode.
+ * <p>The one context where cleanup is guaranteed to be called is when a topology is killed when running Storm in local mode.
*/
void cleanup();
}
diff --git a/storm-client/src/jvm/org/apache/storm/task/IMetricsContext.java b/storm-client/src/jvm/org/apache/storm/task/IMetricsContext.java
index 7ddf078..340cafd 100644
--- a/storm-client/src/jvm/org/apache/storm/task/IMetricsContext.java
+++ b/storm-client/src/jvm/org/apache/storm/task/IMetricsContext.java
@@ -26,18 +26,21 @@
public interface IMetricsContext {
/**
+ * Register metric.
* @deprecated in favor of metrics v2 (the non-deprecated methods on this class)
*/
@Deprecated
<T extends IMetric> T registerMetric(String name, T metric, int timeBucketSizeInSecs);
/**
+ * Register metric.
* @deprecated in favor of metrics v2 (the non-deprecated methods on this class)
*/
@Deprecated
ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs);
/**
+ * Register metric.
* @deprecated in favor of metrics v2 (the non-deprecated methods on this class)
*/
@Deprecated
diff --git a/storm-client/src/jvm/org/apache/storm/task/OutputCollector.java b/storm-client/src/jvm/org/apache/storm/task/OutputCollector.java
index 0d5900e..57ab279 100644
--- a/storm-client/src/jvm/org/apache/storm/task/OutputCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/task/OutputCollector.java
@@ -23,11 +23,11 @@
* and a more restricted form of stream processing, see IBasicBolt and BasicOutputCollector.
*/
public class OutputCollector implements IOutputCollector {
- private IOutputCollector _delegate;
+ private IOutputCollector delegate;
public OutputCollector(IOutputCollector delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
/**
@@ -88,6 +88,11 @@
return emit(Utils.DEFAULT_STREAM_ID, tuple);
}
+ @Override
+ public List<Integer> emit(String streamId, Collection<Tuple> anchors, List<Object> tuple) {
+ return delegate.emit(streamId, anchors, tuple);
+ }
+
/**
* Emits a tuple directly to the specified task id on the specified stream. If the target bolt does not subscribe to this bolt using a
* direct grouping, the tuple will not be sent. If the specified output stream is not declared as direct, or the target bolt subscribes
@@ -121,7 +126,7 @@
* direct grouping, the tuple will not be sent. If the specified output stream is not declared as direct, or the target bolt subscribes
* with a non-direct grouping, an error will occur at runtime. The emitted values must be immutable.
*
- * The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
+ * <p>The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
* defining topologies in Java.
*
* @param taskId the taskId to send the new tuple to
@@ -137,7 +142,7 @@
* direct grouping, the tuple will not be sent. If the specified output stream is not declared as direct, or the target bolt subscribes
* with a non-direct grouping, an error will occur at runtime. The emitted values must be immutable.
*
- * The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
+ * <p>The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
* defining topologies in Java.
*
* @param taskId the taskId to send the new tuple to
@@ -154,10 +159,10 @@
* direct grouping, the tuple will not be sent. If the specified output stream is not declared as direct, or the target bolt subscribes
* with a non-direct grouping, an error will occur at runtime. The emitted values must be immutable.
*
- * The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
+ * <p>The default stream must be declared as direct in the topology definition. See OutputDeclarer#declare for how this is done when
* defining topologies in Java.<
*
- * Note that this method does not use anchors, so downstream failures won't affect the failure status of any spout tuples.
+ * <p>Note that this method does not use anchors, so downstream failures won't affect the failure status of any spout tuples.
*
* @param taskId the taskId to send the new tuple to
* @param tuple the new output tuple from this bolt
@@ -167,23 +172,18 @@
}
@Override
- public List<Integer> emit(String streamId, Collection<Tuple> anchors, List<Object> tuple) {
- return _delegate.emit(streamId, anchors, tuple);
- }
-
- @Override
public void emitDirect(int taskId, String streamId, Collection<Tuple> anchors, List<Object> tuple) {
- _delegate.emitDirect(taskId, streamId, anchors, tuple);
+ delegate.emitDirect(taskId, streamId, anchors, tuple);
}
@Override
public void ack(Tuple input) {
- _delegate.ack(input);
+ delegate.ack(input);
}
@Override
public void fail(Tuple input) {
- _delegate.fail(input);
+ delegate.fail(input);
}
/**
@@ -194,16 +194,16 @@
*/
@Override
public void resetTimeout(Tuple input) {
- _delegate.resetTimeout(input);
+ delegate.resetTimeout(input);
}
@Override
public void reportError(Throwable error) {
- _delegate.reportError(error);
+ delegate.reportError(error);
}
@Override
public void flush() {
- _delegate.flush();
+ delegate.flush();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/task/ShellBolt.java b/storm-client/src/jvm/org/apache/storm/task/ShellBolt.java
index 719b1a8..dfb6383 100644
--- a/storm-client/src/jvm/org/apache/storm/task/ShellBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/task/ShellBolt.java
@@ -18,6 +18,8 @@
package org.apache.storm.task;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@@ -49,50 +51,49 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static java.util.concurrent.TimeUnit.SECONDS;
-
/**
* A bolt that shells out to another process to process tuples. ShellBolt communicates with that process over stdio using a special
* protocol. An ~100 line library is required to implement that protocol, and adapter libraries currently exist for Ruby and Python.
*
- * To run a ShellBolt on a cluster, the scripts that are shelled out to must be in the resources directory within the jar submitted to the
- * master. During development/testing on a local machine, that resources directory just needs to be on the classpath.
+ * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be in the resources directory within the
+ * jar submitted to the master. During development/testing on a local machine, that resources directory just needs to be
+ * on the classpath.
*
- * When creating topologies using the Java API, subclass this bolt and implement the IRichBolt interface to create components for the
+ * <p>When creating topologies using the Java API, subclass this bolt and implement the IRichBolt interface to create components for the
* topology that use other languages. For example:
*
*
- * ```java public class MyBolt extends ShellBolt implements IRichBolt { public MyBolt() { super("python", "mybolt.py"); }
+ * <p>```java public class MyBolt extends ShellBolt implements IRichBolt { public MyBolt() { super("python", "mybolt.py"); }
*
- * public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("field1", "field2")); } } ```
+ * <p>public void declareOutputFields(OutputFieldsDeclarer declarer) { declarer.declare(new Fields("field1", "field2")); } } ```
*/
public class ShellBolt implements IBolt {
public static final String HEARTBEAT_STREAM_ID = "__heartbeat";
public static final Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
private static final long serialVersionUID = -339575186639193348L;
- OutputCollector _collector;
- Map<String, Tuple> _inputs = new ConcurrentHashMap<>();
+ OutputCollector collector;
+ Map<String, Tuple> inputs = new ConcurrentHashMap<>();
- private String[] _command;
+ private String[] command;
private Map<String, String> env = new HashMap<>();
- private ShellLogHandler _logHandler;
- private ShellProcess _process;
- private volatile boolean _running = true;
- private volatile Throwable _exception;
- private ShellBoltMessageQueue _pendingWrites = new ShellBoltMessageQueue();
- private Random _rand;
+ private ShellLogHandler logHandler;
+ private ShellProcess process;
+ private volatile boolean running = true;
+ private volatile Throwable exception;
+ private ShellBoltMessageQueue pendingWrites = new ShellBoltMessageQueue();
+ private Random rand;
- private Thread _readerThread;
- private Thread _writerThread;
+ private Thread readerThread;
+ private Thread writerThread;
- private TopologyContext _context;
+ private TopologyContext context;
private int workerTimeoutMills;
private ScheduledExecutorService heartBeatExecutorService;
private AtomicLong lastHeartbeatTimestamp = new AtomicLong();
private AtomicBoolean sendHeartbeatFlag = new AtomicBoolean(false);
- private boolean _isLocalMode = false;
+ private boolean isLocalMode = false;
private boolean changeDirectory = true;
public ShellBolt(ShellComponent component) {
@@ -100,7 +101,7 @@
}
public ShellBolt(String... command) {
- _command = command;
+ this.command = command;
}
public ShellBolt setEnv(Map<String, String> env) {
@@ -108,6 +109,7 @@
return this;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public boolean shouldChangeChildCWD() {
return changeDirectory;
}
@@ -118,6 +120,7 @@
*
* @param changeDirectory true change the directory (default) false leave the directory the same as the worker process.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public void changeChildCWD(boolean changeDirectory) {
this.changeDirectory = changeDirectory;
}
@@ -126,17 +129,17 @@
public void prepare(Map<String, Object> topoConf, TopologyContext context,
final OutputCollector collector) {
if (ConfigUtils.isLocalMode(topoConf)) {
- _isLocalMode = true;
+ isLocalMode = true;
}
Object maxPending = topoConf.get(Config.TOPOLOGY_SHELLBOLT_MAX_PENDING);
if (maxPending != null) {
- this._pendingWrites = new ShellBoltMessageQueue(((Number) maxPending).intValue());
+ this.pendingWrites = new ShellBoltMessageQueue(((Number) maxPending).intValue());
}
- _rand = new Random();
- _collector = collector;
+ rand = new Random();
+ this.collector = collector;
- _context = context;
+ this.context = context;
if (topoConf.containsKey(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS)) {
workerTimeoutMills = 1000 * ObjectReader.getInt(topoConf.get(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS));
@@ -144,24 +147,24 @@
workerTimeoutMills = 1000 * ObjectReader.getInt(topoConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
}
- _process = new ShellProcess(_command);
+ process = new ShellProcess(command);
if (!env.isEmpty()) {
- _process.setEnv(env);
+ process.setEnv(env);
}
//subprocesses must send their pid first thing
- Number subpid = _process.launch(topoConf, context, changeDirectory);
+ Number subpid = process.launch(topoConf, context, changeDirectory);
LOG.info("Launched subprocess with pid " + subpid);
- _logHandler = ShellUtils.getLogHandler(topoConf);
- _logHandler.setUpContext(ShellBolt.class, _process, _context);
+ logHandler = ShellUtils.getLogHandler(topoConf);
+ logHandler.setUpContext(ShellBolt.class, process, this.context);
// reader
- _readerThread = new Thread(new BoltReaderRunnable());
- _readerThread.start();
+ readerThread = new Thread(new BoltReaderRunnable());
+ readerThread.start();
- _writerThread = new Thread(new BoltWriterRunnable());
- _writerThread.start();
+ writerThread = new Thread(new BoltWriterRunnable());
+ writerThread.start();
LOG.info("Start checking heartbeat...");
setHeartbeat();
@@ -172,17 +175,17 @@
@Override
public void execute(Tuple input) {
- if (_exception != null) {
- throw new RuntimeException(_exception);
+ if (exception != null) {
+ throw new RuntimeException(exception);
}
//just need an id
- String genId = Long.toString(_rand.nextLong());
- _inputs.put(genId, input);
+ String genId = Long.toString(rand.nextLong());
+ inputs.put(genId, input);
try {
BoltMsg boltMsg = createBoltMessage(input, genId);
- _pendingWrites.putBoltMsg(boltMsg);
+ pendingWrites.putBoltMsg(boltMsg);
} catch (InterruptedException e) {
// It's likely that Bolt is shutting down so no need to throw RuntimeException
// just ignore
@@ -201,32 +204,32 @@
@Override
public void cleanup() {
- _running = false;
+ running = false;
heartBeatExecutorService.shutdownNow();
- _writerThread.interrupt();
- _readerThread.interrupt();
- _process.destroy();
- _inputs.clear();
+ writerThread.interrupt();
+ readerThread.interrupt();
+ process.destroy();
+ inputs.clear();
}
private void handleAck(Object id) {
- Tuple acked = _inputs.remove(id);
+ Tuple acked = inputs.remove(id);
if (acked == null) {
throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
}
- _collector.ack(acked);
+ collector.ack(acked);
}
private void handleFail(Object id) {
- Tuple failed = _inputs.remove(id);
+ Tuple failed = inputs.remove(id);
if (failed == null) {
throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
}
- _collector.fail(failed);
+ collector.fail(failed);
}
private void handleError(String msg) {
- _collector.reportError(new Exception("Shell Process Exception: " + msg));
+ collector.reportError(new Exception("Shell Process Exception: " + msg));
}
private void handleEmit(ShellMsg shellMsg) throws InterruptedException {
@@ -234,7 +237,7 @@
List<String> recvAnchors = shellMsg.getAnchors();
if (recvAnchors != null) {
for (String anchor : recvAnchors) {
- Tuple t = _inputs.get(anchor);
+ Tuple t = inputs.get(anchor);
if (t == null) {
throw new RuntimeException("Anchored onto " + anchor + " after ack/fail");
}
@@ -243,12 +246,12 @@
}
if (shellMsg.getTask() == 0) {
- List<Integer> outtasks = _collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());
+ List<Integer> outtasks = collector.emit(shellMsg.getStream(), anchors, shellMsg.getTuple());
if (shellMsg.areTaskIdsNeeded()) {
- _pendingWrites.putTaskIds(outtasks);
+ pendingWrites.putTaskIds(outtasks);
}
} else {
- _collector.emitDirect((int) shellMsg.getTask(),
+ collector.emitDirect((int) shellMsg.getTask(),
shellMsg.getStream(), anchors, shellMsg.getTuple());
}
}
@@ -261,19 +264,19 @@
}
//get metric by name
- IMetric iMetric = _context.getRegisteredMetricByName(name);
- if (iMetric == null) {
+ IMetric metric = context.getRegisteredMetricByName(name);
+ if (metric == null) {
throw new RuntimeException("Could not find metric by name[" + name + "] ");
}
- if (!(iMetric instanceof IShellMetric)) {
+ if (!(metric instanceof IShellMetric)) {
throw new RuntimeException("Metric[" + name + "] is not IShellMetric, can not call by RPC");
}
- IShellMetric iShellMetric = (IShellMetric) iMetric;
+ IShellMetric shellMetric = (IShellMetric) metric;
//call updateMetricFromRPC with params
Object paramsObj = shellMsg.getMetricParams();
try {
- iShellMetric.updateMetricFromRPC(paramsObj);
+ shellMetric.updateMetricFromRPC(paramsObj);
} catch (RuntimeException re) {
throw re;
} catch (Exception e) {
@@ -290,14 +293,14 @@
}
private void die(Throwable exception) {
- String processInfo = _process.getProcessInfoString() + _process.getProcessTerminationInfoString();
- _exception = new RuntimeException(processInfo, exception);
+ String processInfo = process.getProcessInfoString() + process.getProcessTerminationInfoString();
+ this.exception = new RuntimeException(processInfo, exception);
String message = String.format("Halting process: ShellBolt died. Command: %s, ProcessInfo %s",
- Arrays.toString(_command),
+ Arrays.toString(command),
processInfo);
LOG.error(message, exception);
- _collector.reportError(exception);
- if (!_isLocalMode && (_running || (exception instanceof Error))) { //don't exit if not running, unless it is an Error
+ collector.reportError(exception);
+ if (!isLocalMode && (running || (exception instanceof Error))) { //don't exit if not running, unless it is an Error
System.exit(11);
}
}
@@ -328,9 +331,9 @@
private class BoltReaderRunnable implements Runnable {
@Override
public void run() {
- while (_running) {
+ while (running) {
try {
- ShellMsg shellMsg = _process.readShellMsg();
+ ShellMsg shellMsg = process.readShellMsg();
String command = shellMsg.getCommand();
if (command == null) {
@@ -351,7 +354,7 @@
handleError(shellMsg.getMsg());
break;
case "log":
- _logHandler.log(shellMsg);
+ logHandler.log(shellMsg);
break;
case "emit":
handleEmit(shellMsg);
@@ -359,6 +362,9 @@
case "metrics":
handleMetrics(shellMsg);
break;
+ default:
+ throw new IllegalArgumentException(String.format("command %s is not supported",
+ command));
}
} catch (InterruptedException e) {
// It's likely that Bolt is shutting down so no need to die.
@@ -373,21 +379,21 @@
private class BoltWriterRunnable implements Runnable {
@Override
public void run() {
- while (_running) {
+ while (running) {
try {
if (sendHeartbeatFlag.get()) {
LOG.debug("BOLT - sending heartbeat request to subprocess");
- String genId = Long.toString(_rand.nextLong());
- _process.writeBoltMsg(createHeartbeatBoltMessage(genId));
+ String genId = Long.toString(rand.nextLong());
+ process.writeBoltMsg(createHeartbeatBoltMessage(genId));
sendHeartbeatFlag.compareAndSet(true, false);
}
- Object write = _pendingWrites.poll(1, SECONDS);
+ Object write = pendingWrites.poll(1, SECONDS);
if (write instanceof BoltMsg) {
- _process.writeBoltMsg((BoltMsg) write);
+ process.writeBoltMsg((BoltMsg) write);
} else if (write instanceof List<?>) {
- _process.writeTaskIds((List<Integer>) write);
+ process.writeTaskIds((List<Integer>) write);
} else if (write != null) {
throw new RuntimeException(
"Unknown class type to write: " + write.getClass().getName());
diff --git a/storm-client/src/jvm/org/apache/storm/task/TopologyContext.java b/storm-client/src/jvm/org/apache/storm/task/TopologyContext.java
index c6a6371..5a3f4c1 100644
--- a/storm-client/src/jvm/org/apache/storm/task/TopologyContext.java
+++ b/storm-client/src/jvm/org/apache/storm/task/TopologyContext.java
@@ -44,16 +44,16 @@
* A `TopologyContext` is given to bolts and spouts in their `prepare()` and `open()` methods, respectively. This object provides
* information about the component's place within the topology, such as task ids, inputs and outputs, etc.
*
- * The `TopologyContext` is also used to declare `ISubscribedState` objects to synchronize state with StateSpouts this object is subscribed
- * to.
+ * <p>The `TopologyContext` is also used to declare `ISubscribedState` objects to synchronize state with StateSpouts
+ * this object is subscribed to.
*/
public class TopologyContext extends WorkerTopologyContext implements IMetricsContext {
- private final Integer _taskId;
- private final Map<String, Object> _taskData = new HashMap<>();
- private final List<ITaskHook> _hooks = new ArrayList<>();
- private final Map<String, Object> _executorData;
- private final Map<Integer, Map<Integer, Map<String, IMetric>>> _registeredMetrics;
- private final AtomicBoolean _openOrPrepareWasCalled;
+ private final Integer taskId;
+ private final Map<String, Object> taskData = new HashMap<>();
+ private final List<ITaskHook> hooks = new ArrayList<>();
+ private final Map<String, Object> executorData;
+ private final Map<Integer, Map<Integer, Map<String, IMetric>>> registeredMetrics;
+ private final AtomicBoolean openOrPrepareWasCalled;
private final StormMetricRegistry metricRegistry;
// This is updated by the Worker and the topology has shared access to it
private final Map<String, Long> blobToLastKnownVersion;
@@ -80,13 +80,14 @@
componentToStreamToFields, stormId, codeDir, pidDir,
workerPort, workerTasks, defaultResources, userResources);
this.metricRegistry = metricRegistry;
- _taskId = taskId;
- _executorData = executorData;
- _registeredMetrics = registeredMetrics;
- _openOrPrepareWasCalled = openOrPrepareWasCalled;
+ this.taskId = taskId;
+ this.executorData = executorData;
+ this.registeredMetrics = registeredMetrics;
+ this.openOrPrepareWasCalled = openOrPrepareWasCalled;
blobToLastKnownVersion = blobToLastKnownVersionShared;
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private static Map<String, Object> groupingToJSONableMap(Grouping grouping) {
Map<String, Object> groupingMap = new HashMap<>();
groupingMap.put("type", grouping.getSetField().toString());
@@ -99,10 +100,10 @@
/**
* All state from all subscribed state spouts streams will be synced with the provided object.
*
- * It is recommended that your ISubscribedState object is kept as an instance variable of this object. The recommended usage of this
+ * <p>It is recommended that your ISubscribedState object is kept as an instance variable of this object. The recommended usage of this
* method is as follows:
*
- * ```java _myState = context.setAllSubscribedState(new MyState()); ```
+ * <p>```java _myState = context.setAllSubscribedState(new MyState()); ```
*
* @param obj Provided ISubscribedState implementation
* @return Returns the ISubscribedState object provided
@@ -116,9 +117,9 @@
/**
* Synchronizes the default stream from the specified state spout component id with the provided ISubscribedState object.
*
- * The recommended usage of this method is as follows:
+ * <p>The recommended usage of this method is as follows:
*
- * ```java _myState = context.setSubscribedState(componentId, new MyState()); ```
+ * <p>```java _myState = context.setSubscribedState(componentId, new MyState()); ```
*
* @param componentId the id of the StateSpout component to subscribe to
* @param obj Provided ISubscribedState implementation
@@ -131,9 +132,9 @@
/**
* Synchronizes the specified stream from the specified state spout component id with the provided ISubscribedState object.
*
- * The recommended usage of this method is as follows:
+ * <p>The recommended usage of this method is as follows:
*
- * ```java _myState = context.setSubscribedState(componentId, streamId, new MyState()); ```
+ * <p>```java _myState = context.setSubscribedState(componentId, streamId, new MyState()); ```
*
* @param componentId the id of the StateSpout component to subscribe to
* @param streamId the stream to subscribe to
@@ -154,15 +155,16 @@
* @return the task id
*/
public int getThisTaskId() {
- return _taskId;
+ return taskId;
}
/**
+ * Get component id.
* @return the component id for this task. The component id maps to a component id specified for a Spout or Bolt in the topology
* definition.
*/
public String getThisComponentId() {
- return getComponentId(_taskId);
+ return getComponentId(taskId);
}
/**
@@ -246,28 +248,28 @@
}
public void setTaskData(String name, Object data) {
- _taskData.put(name, data);
+ taskData.put(name, data);
}
public Object getTaskData(String name) {
- return _taskData.get(name);
+ return taskData.get(name);
}
public void setExecutorData(String name, Object data) {
- _executorData.put(name, data);
+ executorData.put(name, data);
}
public Object getExecutorData(String name) {
- return _executorData.get(name);
+ return executorData.get(name);
}
public void addTaskHook(ITaskHook hook) {
- hook.prepare(_topoConf, this);
- _hooks.add(hook);
+ hook.prepare(topoConf, this);
+ hooks.add(hook);
}
public List<ITaskHook> getHooks() {
- return _hooks;
+ return hooks;
}
@Override
@@ -318,9 +320,9 @@
@Deprecated
@Override
public <T extends IMetric> T registerMetric(String name, T metric, int timeBucketSizeInSecs) {
- if (_openOrPrepareWasCalled.get()) {
- throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden " +
- "IBolt::prepare() or ISpout::open() method.");
+ if (openOrPrepareWasCalled.get()) {
+ throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden "
+ + "IBolt::prepare() or ISpout::open() method.");
}
if (metric == null) {
@@ -328,25 +330,25 @@
}
if (timeBucketSizeInSecs <= 0) {
- throw new IllegalArgumentException("TopologyContext.registerMetric can only be called with timeBucketSizeInSecs " +
- "greater than or equal to 1 second.");
+ throw new IllegalArgumentException("TopologyContext.registerMetric can only be called with "
+ + "timeBucketSizeInSecs greater than or equal to 1 second.");
}
if (getRegisteredMetricByName(name) != null) {
throw new RuntimeException("The same metric name `" + name + "` was registered twice.");
}
- Map<Integer, Map<Integer, Map<String, IMetric>>> m1 = _registeredMetrics;
+ Map<Integer, Map<Integer, Map<String, IMetric>>> m1 = registeredMetrics;
if (!m1.containsKey(timeBucketSizeInSecs)) {
m1.put(timeBucketSizeInSecs, new HashMap<Integer, Map<String, IMetric>>());
}
Map<Integer, Map<String, IMetric>> m2 = m1.get(timeBucketSizeInSecs);
- if (!m2.containsKey(_taskId)) {
- m2.put(_taskId, new HashMap<String, IMetric>());
+ if (!m2.containsKey(taskId)) {
+ m2.put(taskId, new HashMap<String, IMetric>());
}
- Map<String, IMetric> m3 = m2.get(_taskId);
+ Map<String, IMetric> m3 = m2.get(taskId);
if (m3.containsKey(name)) {
throw new RuntimeException("The same metric name `" + name + "` was registered twice.");
} else {
@@ -356,29 +358,6 @@
return metric;
}
- /**
- * Get component's metric from registered metrics by name. Notice: Normally, one component can only register one metric name once. But
- * now registerMetric has a bug(https://issues.apache.org/jira/browse/STORM-254) cause the same metric name can register twice. So we
- * just return the first metric we meet.
- */
- @Deprecated
- public IMetric getRegisteredMetricByName(String name) {
- IMetric metric = null;
-
- for (Map<Integer, Map<String, IMetric>> taskIdToNameToMetric : _registeredMetrics.values()) {
- Map<String, IMetric> nameToMetric = taskIdToNameToMetric.get(_taskId);
- if (nameToMetric != null) {
- metric = nameToMetric.get(name);
- if (metric != null) {
- //we just return the first metric we meet
- break;
- }
- }
- }
-
- return metric;
- }
-
/*
* Convenience method for registering ReducedMetric.
*/
@@ -397,6 +376,29 @@
return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs);
}
+ /**
+ * Get component's metric from registered metrics by name. Notice: Normally, one component can only register one metric name once. But
+ * now registerMetric has a bug(https://issues.apache.org/jira/browse/STORM-254) cause the same metric name can register twice. So we
+ * just return the first metric we meet.
+ */
+ @Deprecated
+ public IMetric getRegisteredMetricByName(String name) {
+ IMetric metric = null;
+
+ for (Map<Integer, Map<String, IMetric>> taskIdToNameToMetric : registeredMetrics.values()) {
+ Map<String, IMetric> nameToMetric = taskIdToNameToMetric.get(taskId);
+ if (nameToMetric != null) {
+ metric = nameToMetric.get(name);
+ if (metric != null) {
+ //we just return the first metric we meet
+ break;
+ }
+ }
+ }
+
+ return metric;
+ }
+
@Override
public Timer registerTimer(String name) {
return metricRegistry.registry().timer(metricName(name));
diff --git a/storm-client/src/jvm/org/apache/storm/task/WorkerTopologyContext.java b/storm-client/src/jvm/org/apache/storm/task/WorkerTopologyContext.java
index 6c6e5a4..36ef800 100644
--- a/storm-client/src/jvm/org/apache/storm/task/WorkerTopologyContext.java
+++ b/storm-client/src/jvm/org/apache/storm/task/WorkerTopologyContext.java
@@ -24,12 +24,12 @@
public class WorkerTopologyContext extends GeneralTopologyContext {
public static final String SHARED_EXECUTOR = "executor";
- Map<String, Object> _userResources;
- Map<String, Object> _defaultResources;
- private Integer _workerPort;
- private List<Integer> _workerTasks;
- private String _codeDir;
- private String _pidDir;
+ Map<String, Object> userResources;
+ Map<String, Object> defaultResources;
+ private Integer workerPort;
+ private List<Integer> workerTasks;
+ private String codeDir;
+ private String pidDir;
private AtomicReference<Map<Integer, NodeInfo>> taskToNodePort;
private String assignmentId;
@@ -50,20 +50,20 @@
String assignmentId
) {
super(topology, topoConf, taskToComponent, componentToSortedTasks, componentToStreamToFields, stormId);
- _codeDir = codeDir;
- _defaultResources = defaultResources;
- _userResources = userResources;
+ this.codeDir = codeDir;
+ this.defaultResources = defaultResources;
+ this.userResources = userResources;
try {
if (pidDir != null) {
- _pidDir = new File(pidDir).getCanonicalPath();
+ this.pidDir = new File(pidDir).getCanonicalPath();
} else {
- _pidDir = null;
+ this.pidDir = null;
}
} catch (IOException e) {
- throw new RuntimeException("Could not get canonical path for " + _pidDir, e);
+ throw new RuntimeException("Could not get canonical path for " + this.pidDir, e);
}
- _workerPort = workerPort;
- _workerTasks = workerTasks;
+ this.workerPort = workerPort;
+ this.workerTasks = workerTasks;
this.taskToNodePort = taskToNodePort;
this.assignmentId = assignmentId;
@@ -90,11 +90,11 @@
* Gets all the task ids that are running in this worker process (including the task for this task).
*/
public List<Integer> getThisWorkerTasks() {
- return _workerTasks;
+ return workerTasks;
}
public Integer getThisWorkerPort() {
- return _workerPort;
+ return workerPort;
}
public String getThisWorkerHost() {
@@ -102,7 +102,7 @@
}
/**
- * Get a map from task Id to NodePort
+ * Get a map from task Id to NodePort.
*
* @return a map from task To NodePort
*/
@@ -115,22 +115,23 @@
* implemented in other languages, such as Ruby or Python.
*/
public String getCodeDir() {
- return _codeDir;
+ return codeDir;
}
/**
* If this task spawns any subprocesses, those subprocesses must immediately write their PID to this directory on the local filesystem
* to ensure that Storm properly destroys that process when the worker is shutdown.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public String getPIDDir() {
- return _pidDir;
+ return pidDir;
}
public Object getResource(String name) {
- return _userResources.get(name);
+ return userResources.get(name);
}
public ExecutorService getSharedExecutor() {
- return (ExecutorService) _defaultResources.get(SHARED_EXECUTOR);
+ return (ExecutorService) defaultResources.get(SHARED_EXECUTOR);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/AckFailMapTracker.java b/storm-client/src/jvm/org/apache/storm/testing/AckFailMapTracker.java
index ce26183..2bdff61 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/AckFailMapTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/AckFailMapTracker.java
@@ -18,30 +18,30 @@
public class AckFailMapTracker implements AckFailDelegate {
- String _acked;
- String _failed;
+ private String acked;
+ private String failed;
public AckFailMapTracker() {
- _acked = RegisteredGlobalState.registerState(new HashSet());
- _failed = RegisteredGlobalState.registerState(new HashSet());
+ acked = RegisteredGlobalState.registerState(new HashSet());
+ failed = RegisteredGlobalState.registerState(new HashSet());
}
public boolean isAcked(Object id) {
- return ((Set) RegisteredGlobalState.getState(_acked)).contains(id);
+ return ((Set) RegisteredGlobalState.getState(acked)).contains(id);
}
public boolean isFailed(Object id) {
- return ((Set) RegisteredGlobalState.getState(_failed)).contains(id);
+ return ((Set) RegisteredGlobalState.getState(failed)).contains(id);
}
@Override
public void ack(Object id) {
- ((Set) RegisteredGlobalState.getState(_acked)).add(id);
+ ((Set) RegisteredGlobalState.getState(acked)).add(id);
}
@Override
public void fail(Object id) {
- ((Set) RegisteredGlobalState.getState(_failed)).add(id);
+ ((Set) RegisteredGlobalState.getState(failed)).add(id);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/AckTracker.java b/storm-client/src/jvm/org/apache/storm/testing/AckTracker.java
index bb448e6..e62dee2 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/AckTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/AckTracker.java
@@ -20,16 +20,16 @@
public class AckTracker implements AckFailDelegate {
private static Map<String, AtomicInteger> acks = new ConcurrentHashMap<String, AtomicInteger>();
- private String _id;
+ private String id;
public AckTracker() {
- _id = UUID.randomUUID().toString();
- acks.put(_id, new AtomicInteger(0));
+ id = UUID.randomUUID().toString();
+ acks.put(id, new AtomicInteger(0));
}
@Override
public void ack(Object id) {
- acks.get(_id).incrementAndGet();
+ acks.get(this.id).incrementAndGet();
}
@Override
@@ -37,11 +37,11 @@
}
public int getNumAcks() {
- return acks.get(_id).intValue();
+ return acks.get(id).intValue();
}
public void resetNumAcks() {
- acks.get(_id).set(0);
+ acks.get(id).set(0);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/AlternateRackDNSToSwitchMapping.java b/storm-client/src/jvm/org/apache/storm/testing/AlternateRackDNSToSwitchMapping.java
index d4b1dd5..411c382 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/AlternateRackDNSToSwitchMapping.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/AlternateRackDNSToSwitchMapping.java
@@ -23,6 +23,7 @@
/**
* This class implements the {@link DNSToSwitchMapping} interface It alternates bewteen RACK1 and RACK2 for the hosts.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public final class AlternateRackDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
private Map<String, String> mappingCache = new ConcurrentHashMap<String, String>();
diff --git a/storm-client/src/jvm/org/apache/storm/testing/BoltTracker.java b/storm-client/src/jvm/org/apache/storm/testing/BoltTracker.java
index 5a57536..d6ff164 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/BoltTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/BoltTracker.java
@@ -19,16 +19,16 @@
public class BoltTracker extends NonRichBoltTracker implements IRichBolt {
- IRichBolt _richDelegate;
+ IRichBolt richDelegate;
public BoltTracker(IRichBolt delegate, String id) {
super(delegate, id);
- _richDelegate = delegate;
+ richDelegate = delegate;
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _richDelegate.declareOutputFields(declarer);
+ richDelegate.declareOutputFields(declarer);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/CompletableSpout.java b/storm-client/src/jvm/org/apache/storm/testing/CompletableSpout.java
index 54fef63..57d24bc 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/CompletableSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/CompletableSpout.java
@@ -14,21 +14,22 @@
public interface CompletableSpout {
/**
+ * Check whether spout is exhausted.
* @return true if all the tuples have been completed else false.
*/
- public boolean isExhausted();
+ boolean isExhausted();
/**
- * Cleanup any global state kept
+ * Cleanup any global state kept.
*/
- default public void clean() {
+ default void clean() {
//NOOP
}
/**
- * Prepare the spout (globally) before starting the topology
+ * Prepare the spout (globally) before starting the topology.
*/
- default public void startup() {
+ default void startup() {
//NOOP
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/FeederSpout.java b/storm-client/src/jvm/org/apache/storm/testing/FeederSpout.java
index 5cce40c..32a0b25 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/FeederSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/FeederSpout.java
@@ -26,22 +26,22 @@
public class FeederSpout extends BaseRichSpout {
- private int _id;
- private Fields _outFields;
- private SpoutOutputCollector _collector;
- private AckFailDelegate _ackFailDelegate;
+ private int id;
+ private Fields outFields;
+ private SpoutOutputCollector collector;
+ private AckFailDelegate ackFailDelegate;
public FeederSpout(List<String> outFields) {
this(new Fields(outFields));
}
public FeederSpout(Fields outFields) {
- _id = InprocMessaging.acquireNewPort();
- _outFields = outFields;
+ id = InprocMessaging.acquireNewPort();
+ this.outFields = outFields;
}
public void setAckFailDelegate(AckFailDelegate d) {
- _ackFailDelegate = d;
+ ackFailDelegate = d;
}
public void feed(List<Object> tuple) {
@@ -49,20 +49,20 @@
}
public void feed(List<Object> tuple, Object msgId) {
- InprocMessaging.sendMessage(_id, new Values(tuple, msgId));
+ InprocMessaging.sendMessage(id, new Values(tuple, msgId));
}
public void feedNoWait(List<Object> tuple, Object msgId) {
- InprocMessaging.sendMessageNoWait(_id, new Values(tuple, msgId));
+ InprocMessaging.sendMessageNoWait(id, new Values(tuple, msgId));
}
public void waitForReader() {
- InprocMessaging.waitForReader(_id);
+ InprocMessaging.waitForReader(id);
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
@@ -72,32 +72,32 @@
@Override
public void nextTuple() {
- List<Object> toEmit = (List<Object>) InprocMessaging.pollMessage(_id);
+ List<Object> toEmit = (List<Object>) InprocMessaging.pollMessage(id);
if (toEmit != null) {
List<Object> tuple = (List<Object>) toEmit.get(0);
Object msgId = toEmit.get(1);
- _collector.emit(tuple, msgId);
+ collector.emit(tuple, msgId);
}
}
@Override
public void ack(Object msgId) {
- if (_ackFailDelegate != null) {
- _ackFailDelegate.ack(msgId);
+ if (ackFailDelegate != null) {
+ ackFailDelegate.ack(msgId);
}
}
@Override
public void fail(Object msgId) {
- if (_ackFailDelegate != null) {
- _ackFailDelegate.fail(msgId);
+ if (ackFailDelegate != null) {
+ ackFailDelegate.fail(msgId);
}
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(_outFields);
+ declarer.declare(outFields);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/FixedTupleSpout.java b/storm-client/src/jvm/org/apache/storm/testing/FixedTupleSpout.java
index 8404f86..c58dd93 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/FixedTupleSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/FixedTupleSpout.java
@@ -12,6 +12,8 @@
package org.apache.storm.testing;
+import static org.apache.storm.utils.Utils.get;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -23,32 +25,30 @@
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.tuple.Fields;
-import static org.apache.storm.utils.Utils.get;
-
public class FixedTupleSpout implements IRichSpout, CompletableSpout {
private static final Map<String, Integer> acked = new HashMap<String, Integer>();
private static final Map<String, Integer> failed = new HashMap<String, Integer>();
- private List<FixedTuple> _tuples;
- private SpoutOutputCollector _collector;
- private TopologyContext _context;
- private List<FixedTuple> _serveTuples;
- private Map<String, FixedTuple> _pending;
- private String _id;
- private Fields _fields;
+ private List<FixedTuple> tuples;
+ private SpoutOutputCollector collector;
+ private TopologyContext context;
+ private List<FixedTuple> serveTuples;
+ private Map<String, FixedTuple> pending;
+ private String id;
+ private Fields fields;
public FixedTupleSpout(List tuples) {
this(tuples, (Fields) null);
}
public FixedTupleSpout(List tuples, Fields fields) {
- _id = UUID.randomUUID().toString();
+ id = UUID.randomUUID().toString();
synchronized (acked) {
- acked.put(_id, 0);
+ acked.put(id, 0);
}
synchronized (failed) {
- failed.put(_id, 0);
+ failed.put(id, 0);
}
- _tuples = new ArrayList<FixedTuple>();
+ this.tuples = new ArrayList<FixedTuple>();
for (Object o : tuples) {
FixedTuple ft;
if (o instanceof FixedTuple) {
@@ -56,9 +56,9 @@
} else {
ft = new FixedTuple((List) o);
}
- _tuples.add(ft);
+ this.tuples.add(ft);
}
- _fields = fields;
+ this.fields = fields;
}
public static int getNumAcked(String stormId) {
@@ -79,7 +79,7 @@
}
public List<FixedTuple> getSourceTuples() {
- return _tuples;
+ return tuples;
}
public int getCompleted() {
@@ -87,26 +87,26 @@
int failedAmt;
synchronized (acked) {
- ackedAmt = acked.get(_id);
+ ackedAmt = acked.get(id);
}
synchronized (failed) {
- failedAmt = failed.get(_id);
+ failedAmt = failed.get(id);
}
return ackedAmt + failedAmt;
}
public void cleanup() {
synchronized (acked) {
- acked.remove(_id);
+ acked.remove(id);
}
synchronized (failed) {
- failed.remove(_id);
+ failed.remove(id);
}
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _context = context;
+ this.context = context;
List<Integer> tasks = context.getComponentTasks(context.getThisComponentId());
int startIndex;
for (startIndex = 0; startIndex < tasks.size(); startIndex++) {
@@ -114,11 +114,11 @@
break;
}
}
- _collector = collector;
- _pending = new HashMap<String, FixedTuple>();
- _serveTuples = new ArrayList<FixedTuple>();
- for (int i = startIndex; i < _tuples.size(); i += tasks.size()) {
- _serveTuples.add(_tuples.get(i));
+ this.collector = collector;
+ pending = new HashMap<String, FixedTuple>();
+ serveTuples = new ArrayList<FixedTuple>();
+ for (int i = startIndex; i < tuples.size(); i += tasks.size()) {
+ serveTuples.add(tuples.get(i));
}
}
@@ -128,27 +128,27 @@
@Override
public void nextTuple() {
- if (_serveTuples.size() > 0) {
- FixedTuple ft = _serveTuples.remove(0);
+ if (serveTuples.size() > 0) {
+ FixedTuple ft = serveTuples.remove(0);
String id = UUID.randomUUID().toString();
- _pending.put(id, ft);
- _collector.emit(ft.stream, ft.values, id);
+ pending.put(id, ft);
+ collector.emit(ft.stream, ft.values, id);
}
}
@Override
public void ack(Object msgId) {
synchronized (acked) {
- int curr = get(acked, _id, 0);
- acked.put(_id, curr + 1);
+ int curr = get(acked, id, 0);
+ acked.put(id, curr + 1);
}
}
@Override
public void fail(Object msgId) {
synchronized (failed) {
- int curr = get(failed, _id, 0);
- failed.put(_id, curr + 1);
+ int curr = get(failed, id, 0);
+ failed.put(id, curr + 1);
}
}
@@ -162,8 +162,8 @@
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- if (_fields != null) {
- declarer.declare(_fields);
+ if (fields != null) {
+ declarer.declare(fields);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/ForwardingMetricsConsumer.java b/storm-client/src/jvm/org/apache/storm/testing/ForwardingMetricsConsumer.java
index 859a640..7fc863e 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/ForwardingMetricsConsumer.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/ForwardingMetricsConsumer.java
@@ -61,16 +61,18 @@
@Override
public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
StringBuilder sb = new StringBuilder();
- String header = taskInfo.timestamp + "\t" +
- taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort + "\t" +
- taskInfo.srcTaskId + "\t" + taskInfo.srcComponentId + "\t";
+ String header = taskInfo.timestamp + "\t"
+ + taskInfo.srcWorkerHost + ":"
+ + taskInfo.srcWorkerPort + "\t"
+ + taskInfo.srcTaskId + "\t"
+ + taskInfo.srcComponentId + "\t";
sb.append(header);
for (DataPoint p : dataPoints) {
sb.delete(header.length(), sb.length());
sb.append(p.name)
- .append("\t")
- .append(p.value)
- .append("\n");
+ .append("\t")
+ .append(p.value)
+ .append("\n");
try {
out.write(sb.toString().getBytes());
out.flush();
diff --git a/storm-client/src/jvm/org/apache/storm/testing/IdentityBolt.java b/storm-client/src/jvm/org/apache/storm/testing/IdentityBolt.java
index 95f5f8a..93c5e95 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/IdentityBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/IdentityBolt.java
@@ -19,10 +19,10 @@
import org.apache.storm.tuple.Tuple;
public class IdentityBolt extends BaseBasicBolt {
- Fields _fields;
+ Fields fields;
public IdentityBolt(Fields fields) {
- _fields = fields;
+ this.fields = fields;
}
@Override
@@ -32,6 +32,6 @@
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(_fields);
+ declarer.declare(fields);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/MkClusterParam.java b/storm-client/src/jvm/org/apache/storm/testing/MkClusterParam.java
index 5a8df3e..55612fd 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/MkClusterParam.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/MkClusterParam.java
@@ -15,7 +15,7 @@
import java.util.Map;
/**
- * The param arg for `Testing.withSimulatedTimeCluster`, `Testing.withTrackedCluster` and `Testing.withLocalCluster`
+ * The param arg for `Testing.withSimulatedTimeCluster`, `Testing.withTrackedCluster` and `Testing.withLocalCluster`.
*/
public class MkClusterParam {
/**
@@ -23,11 +23,11 @@
*/
private Integer supervisors;
/**
- * count of port for each supervisor
+ * count of port for each supervisor.
*/
private Integer portsPerSupervisor;
/**
- * cluster config
+ * cluster config.
*/
private Map<String, Object> daemonConf;
@@ -63,7 +63,7 @@
/**
* When nimbusDaemon is true, the local cluster will be started with a Nimbus Thrift server, allowing communication through for example
- * org.apache.storm.utils.NimbusClient
+ * org.apache.storm.utils.NimbusClient.
*/
public void setNimbusDaemon(Boolean nimbusDaemon) {
this.nimbusDaemon = nimbusDaemon;
diff --git a/storm-client/src/jvm/org/apache/storm/testing/NGrouping.java b/storm-client/src/jvm/org/apache/storm/testing/NGrouping.java
index 53eb276..3135b32 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/NGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/NGrouping.java
@@ -20,26 +20,26 @@
import org.apache.storm.task.WorkerTopologyContext;
public class NGrouping implements CustomStreamGrouping {
- int _n;
- List<Integer> _outTasks;
+ int number;
+ List<Integer> outTasks;
public NGrouping(Integer n) {
- _n = n;
+ number = n;
}
@Override
public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
targetTasks = new ArrayList<Integer>(targetTasks);
Collections.sort(targetTasks);
- _outTasks = new ArrayList<Integer>();
- for (int i = 0; i < _n; i++) {
- _outTasks.add(targetTasks.get(i));
+ outTasks = new ArrayList<Integer>();
+ for (int i = 0; i < number; i++) {
+ outTasks.add(targetTasks.get(i));
}
}
@Override
public List<Integer> chooseTasks(int taskId, List<Object> values) {
- return _outTasks;
+ return outTasks;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/NonRichBoltTracker.java b/storm-client/src/jvm/org/apache/storm/testing/NonRichBoltTracker.java
index f5d0327..815ba71 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/NonRichBoltTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/NonRichBoltTracker.java
@@ -22,28 +22,28 @@
public class NonRichBoltTracker implements IBolt {
- IBolt _delegate;
- String _trackId;
+ IBolt delegate;
+ String trackId;
public NonRichBoltTracker(IBolt delegate, String id) {
- _delegate = delegate;
- _trackId = id;
+ this.delegate = delegate;
+ trackId = id;
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _delegate.prepare(topoConf, context, collector);
+ delegate.prepare(topoConf, context, collector);
}
@Override
public void execute(Tuple input) {
- _delegate.execute(input);
- Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(_trackId);
+ delegate.execute(input);
+ Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(trackId);
((AtomicInteger) stats.get("processed")).incrementAndGet();
}
@Override
public void cleanup() {
- _delegate.cleanup();
+ delegate.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/PrepareBatchBolt.java b/storm-client/src/jvm/org/apache/storm/testing/PrepareBatchBolt.java
index df7da85..35c1b09 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/PrepareBatchBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/PrepareBatchBolt.java
@@ -23,15 +23,15 @@
public class PrepareBatchBolt extends BaseBasicBolt {
- Fields _outFields;
+ Fields outFields;
public PrepareBatchBolt(Fields outFields) {
- _outFields = outFields;
+ this.outFields = outFields;
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(_outFields);
+ declarer.declare(outFields);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsBolt.java b/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsBolt.java
index 48e08f1..bdd4edf 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsBolt.java
@@ -41,8 +41,8 @@
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
super.prepare(topoConf, context, collector);
- CountShellMetric cMetric = new CountShellMetric();
- context.registerMetric("my-custom-shell-metric", cMetric, 5);
+ CountShellMetric countShellMetric = new CountShellMetric();
+ context.registerMetric("my-custom-shell-metric", countShellMetric, 5);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsSpout.java b/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsSpout.java
index e0795cb..f7ecb51 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/PythonShellMetricsSpout.java
@@ -42,8 +42,8 @@
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
super.open(conf, context, collector);
- CountShellMetric cMetric = new CountShellMetric();
- context.registerMetric("my-custom-shellspout-metric", cMetric, 5);
+ CountShellMetric countShellMetric = new CountShellMetric();
+ context.registerMetric("my-custom-shellspout-metric", countShellMetric, 5);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/SpoutTracker.java b/storm-client/src/jvm/org/apache/storm/testing/SpoutTracker.java
index 88a48b6..39ef158 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/SpoutTracker.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/SpoutTracker.java
@@ -25,92 +25,92 @@
public class SpoutTracker extends BaseRichSpout {
- IRichSpout _delegate;
- SpoutTrackOutputCollector _tracker;
- String _trackId;
+ IRichSpout delegate;
+ SpoutTrackOutputCollector tracker;
+ String trackId;
public SpoutTracker(IRichSpout delegate, String trackId) {
- _delegate = delegate;
- _trackId = trackId;
+ this.delegate = delegate;
+ this.trackId = trackId;
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _tracker = new SpoutTrackOutputCollector(collector);
- _delegate.open(conf, context, new SpoutOutputCollector(_tracker));
+ tracker = new SpoutTrackOutputCollector(collector);
+ delegate.open(conf, context, new SpoutOutputCollector(tracker));
}
@Override
public void close() {
- _delegate.close();
+ delegate.close();
}
@Override
public void nextTuple() {
- _delegate.nextTuple();
+ delegate.nextTuple();
}
@Override
public void ack(Object msgId) {
- _delegate.ack(msgId);
- Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(_trackId);
+ delegate.ack(msgId);
+ Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(trackId);
((AtomicInteger) stats.get("processed")).incrementAndGet();
}
@Override
public void fail(Object msgId) {
- _delegate.fail(msgId);
- Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(_trackId);
+ delegate.fail(msgId);
+ Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(trackId);
((AtomicInteger) stats.get("processed")).incrementAndGet();
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _delegate.declareOutputFields(declarer);
+ delegate.declareOutputFields(declarer);
}
private class SpoutTrackOutputCollector implements ISpoutOutputCollector {
public int transferred = 0;
public int emitted = 0;
- public SpoutOutputCollector _collector;
+ public SpoutOutputCollector collector;
public SpoutTrackOutputCollector(SpoutOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
private void recordSpoutEmit() {
- Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(_trackId);
+ Map<String, Object> stats = (Map<String, Object>) RegisteredGlobalState.getState(trackId);
((AtomicInteger) stats.get("spout-emitted")).incrementAndGet();
}
@Override
public List<Integer> emit(String streamId, List<Object> tuple, Object messageId) {
- List<Integer> ret = _collector.emit(streamId, tuple, messageId);
+ List<Integer> ret = collector.emit(streamId, tuple, messageId);
recordSpoutEmit();
return ret;
}
@Override
public void emitDirect(int taskId, String streamId, List<Object> tuple, Object messageId) {
- _collector.emitDirect(taskId, streamId, tuple, messageId);
+ collector.emitDirect(taskId, streamId, tuple, messageId);
recordSpoutEmit();
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void reportError(Throwable error) {
- _collector.reportError(error);
+ collector.reportError(error);
}
@Override
public long getPendingCount() {
- return _collector.getPendingCount();
+ return collector.getPendingCount();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestAggregatesCounter.java b/storm-client/src/jvm/org/apache/storm/testing/TestAggregatesCounter.java
index 8fff86d..47fdae5 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestAggregatesCounter.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestAggregatesCounter.java
@@ -12,6 +12,8 @@
package org.apache.storm.testing;
+import static org.apache.storm.utils.Utils.tuple;
+
import java.util.HashMap;
import java.util.Map;
import org.apache.storm.task.OutputCollector;
@@ -23,32 +25,30 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.utils.Utils.tuple;
-
public class TestAggregatesCounter extends BaseRichBolt {
public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class);
- Map<String, Integer> _counts;
- OutputCollector _collector;
+ Map<String, Integer> counts;
+ OutputCollector collector;
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _collector = collector;
- _counts = new HashMap<String, Integer>();
+ this.collector = collector;
+ counts = new HashMap<String, Integer>();
}
@Override
public void execute(Tuple input) {
String word = (String) input.getValues().get(0);
int count = (Integer) input.getValues().get(1);
- _counts.put(word, count);
+ counts.put(word, count);
int globalCount = 0;
- for (String w : _counts.keySet()) {
- globalCount += _counts.get(w);
+ for (String w : counts.keySet()) {
+ globalCount += counts.get(w);
}
- _collector.emit(tuple(globalCount));
- _collector.ack(input);
+ collector.emit(tuple(globalCount));
+ collector.ack(input);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestConfBolt.java b/storm-client/src/jvm/org/apache/storm/testing/TestConfBolt.java
index 2337c1b..51cb45e 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestConfBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestConfBolt.java
@@ -23,20 +23,20 @@
public class TestConfBolt extends BaseBasicBolt {
- Map<String, Object> _componentConf;
- Map<String, Object> _conf;
+ Map<String, Object> componentConf;
+ Map<String, Object> conf;
public TestConfBolt() {
this(null);
}
public TestConfBolt(Map<String, Object> componentConf) {
- _componentConf = componentConf;
+ this.componentConf = componentConf;
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context) {
- _conf = conf;
+ this.conf = conf;
}
@Override
@@ -47,11 +47,11 @@
@Override
public void execute(Tuple input, BasicOutputCollector collector) {
String name = input.getString(0);
- collector.emit(new Values(name, _conf.get(name)));
+ collector.emit(new Values(name, conf.get(name)));
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _componentConf;
+ return componentConf;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestEventLogSpout.java b/storm-client/src/jvm/org/apache/storm/testing/TestEventLogSpout.java
index 9418fbb..4f4df3d 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestEventLogSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestEventLogSpout.java
@@ -12,6 +12,8 @@
package org.apache.storm.testing;
+import static org.apache.storm.utils.Utils.get;
+
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
@@ -24,13 +26,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.utils.Utils.get;
-
public class TestEventLogSpout extends BaseRichSpout implements CompletableSpout {
private static final Map<String, Integer> acked = new HashMap<String, Integer>();
private static final Map<String, Integer> failed = new HashMap<String, Integer>();
public static Logger LOG = LoggerFactory.getLogger(TestEventLogSpout.class);
- SpoutOutputCollector _collector;
+ SpoutOutputCollector collector;
private String uid;
private long totalCount;
private long eventId = 0;
@@ -64,7 +64,7 @@
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
this.source = context.getThisTaskId();
long taskCount = context.getComponentTasks(context.getThisComponentId()).size();
myCount = totalCount / taskCount;
@@ -107,7 +107,7 @@
public void nextTuple() {
if (eventId < myCount) {
eventId++;
- _collector.emit(new Values(source, eventId), eventId);
+ collector.emit(new Values(source, eventId), eventId);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestEventOrderCheckBolt.java b/storm-client/src/jvm/org/apache/storm/testing/TestEventOrderCheckBolt.java
index 320620f..3511e4e 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestEventOrderCheckBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestEventOrderCheckBolt.java
@@ -26,14 +26,14 @@
public class TestEventOrderCheckBolt extends BaseRichBolt {
public static Logger LOG = LoggerFactory.getLogger(TestEventOrderCheckBolt.class);
- OutputCollector _collector;
+ OutputCollector collector;
Map<Integer, Long> recentEventId = new HashMap<Integer, Long>();
- private int _count;
+ private int count;
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _collector = collector;
- _count = 0;
+ this.collector = collector;
+ count = 0;
}
@Override
@@ -46,11 +46,11 @@
String error = "Error: event id is not in strict order! event source Id: "
+ sourceId + ", last event Id: " + recentEvent + ", current event Id: " + eventId;
- _collector.emit(input, new Values(error));
+ collector.emit(input, new Values(error));
}
recentEventId.put(sourceId, eventId);
- _collector.ack(input);
+ collector.ack(input);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestGlobalCount.java b/storm-client/src/jvm/org/apache/storm/testing/TestGlobalCount.java
index 245b82c..7acc4e1 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestGlobalCount.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestGlobalCount.java
@@ -26,20 +26,20 @@
public class TestGlobalCount extends BaseRichBolt {
public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class);
- OutputCollector _collector;
- private int _count;
+ OutputCollector collector;
+ private int count;
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _collector = collector;
- _count = 0;
+ this.collector = collector;
+ count = 0;
}
@Override
public void execute(Tuple input) {
- _count++;
- _collector.emit(input, new Values(_count));
- _collector.ack(input);
+ count++;
+ collector.emit(input, new Values(count));
+ collector.ack(input);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestPlannerSpout.java b/storm-client/src/jvm/org/apache/storm/testing/TestPlannerSpout.java
index 999f068..bec73af 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestPlannerSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestPlannerSpout.java
@@ -24,12 +24,12 @@
public class TestPlannerSpout extends BaseRichSpout {
- boolean _isDistributed;
- Fields _outFields;
+ boolean isDistributed;
+ Fields outFields;
public TestPlannerSpout(Fields outFields, boolean isDistributed) {
- _isDistributed = isDistributed;
- _outFields = outFields;
+ this.isDistributed = isDistributed;
+ this.outFields = outFields;
}
public TestPlannerSpout(boolean isDistributed) {
@@ -41,7 +41,7 @@
}
public Fields getOutputFields() {
- return _outFields;
+ return outFields;
}
@@ -78,7 +78,7 @@
@Override
public Map<String, Object> getComponentConfiguration() {
Map<String, Object> ret = new HashMap<String, Object>();
- if (!_isDistributed) {
+ if (!isDistributed) {
ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1);
}
return ret;
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestWordCounter.java b/storm-client/src/jvm/org/apache/storm/testing/TestWordCounter.java
index e10e066..1f88c51 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestWordCounter.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestWordCounter.java
@@ -12,6 +12,8 @@
package org.apache.storm.testing;
+import static org.apache.storm.utils.Utils.tuple;
+
import java.util.HashMap;
import java.util.Map;
import org.apache.storm.task.TopologyContext;
@@ -23,17 +25,15 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.utils.Utils.tuple;
-
public class TestWordCounter extends BaseBasicBolt {
public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class);
- Map<String, Integer> _counts;
+ Map<String, Integer> counts;
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context) {
- _counts = new HashMap<String, Integer>();
+ counts = new HashMap<String, Integer>();
}
protected String getTupleValue(Tuple t, int idx) {
@@ -44,11 +44,11 @@
public void execute(Tuple input, BasicOutputCollector collector) {
String word = getTupleValue(input, 0);
int count = 0;
- if (_counts.containsKey(word)) {
- count = _counts.get(word);
+ if (counts.containsKey(word)) {
+ count = counts.get(word);
}
count++;
- _counts.put(word, count);
+ counts.put(word, count);
collector.emit(tuple(word, count));
}
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TestWordSpout.java b/storm-client/src/jvm/org/apache/storm/testing/TestWordSpout.java
index 2c86c68..0d179ce 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TestWordSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TestWordSpout.java
@@ -29,20 +29,20 @@
public class TestWordSpout extends BaseRichSpout {
public static Logger LOG = LoggerFactory.getLogger(TestWordSpout.class);
- boolean _isDistributed;
- SpoutOutputCollector _collector;
+ boolean isDistributed;
+ SpoutOutputCollector collector;
public TestWordSpout() {
this(true);
}
public TestWordSpout(boolean isDistributed) {
- _isDistributed = isDistributed;
+ this.isDistributed = isDistributed;
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
@@ -56,7 +56,7 @@
final String[] words = new String[]{ "nathan", "mike", "jackson", "golda", "bertels" };
final Random rand = new Random();
final String word = words[rand.nextInt(words.length)];
- _collector.emit(new Values(word));
+ collector.emit(new Values(word));
}
@Override
@@ -76,7 +76,7 @@
@Override
public Map<String, Object> getComponentConfiguration() {
- if (!_isDistributed) {
+ if (!isDistributed) {
Map<String, Object> ret = new HashMap<String, Object>();
ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1);
return ret;
diff --git a/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java b/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java
index 13b5b4e..19bdf86 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java
@@ -27,32 +27,32 @@
public class TupleCaptureBolt implements IRichBolt {
public static final transient Map<String, Map<String, List<FixedTuple>>> emitted_tuples = new HashMap<>();
- private String _name;
- private OutputCollector _collector;
+ private String name;
+ private OutputCollector collector;
public TupleCaptureBolt() {
- _name = UUID.randomUUID().toString();
- emitted_tuples.put(_name, new HashMap<String, List<FixedTuple>>());
+ name = UUID.randomUUID().toString();
+ emitted_tuples.put(name, new HashMap<String, List<FixedTuple>>());
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
public void execute(Tuple input) {
String component = input.getSourceComponent();
- Map<String, List<FixedTuple>> captured = emitted_tuples.get(_name);
+ Map<String, List<FixedTuple>> captured = emitted_tuples.get(name);
if (!captured.containsKey(component)) {
captured.put(component, new ArrayList<FixedTuple>());
}
captured.get(component).add(new FixedTuple(input.getSourceStreamId(), input.getValues()));
- _collector.ack(input);
+ collector.ack(input);
}
public Map<String, List<FixedTuple>> getResults() {
- return emitted_tuples.get(_name);
+ return emitted_tuples.get(name);
}
@Override
@@ -60,12 +60,12 @@
}
public Map<String, List<FixedTuple>> getAndRemoveResults() {
- return emitted_tuples.remove(_name);
+ return emitted_tuples.remove(name);
}
public Map<String, List<FixedTuple>> getAndClearResults() {
- Map<String, List<FixedTuple>> ret = new HashMap<>(emitted_tuples.get(_name));
- emitted_tuples.get(_name).clear();
+ Map<String, List<FixedTuple>> ret = new HashMap<>(emitted_tuples.get(name));
+ emitted_tuples.get(name).clear();
return ret;
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/BaseConfigurationDeclarer.java b/storm-client/src/jvm/org/apache/storm/topology/BaseConfigurationDeclarer.java
index 15cb84f..830b2ed 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/BaseConfigurationDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/BaseConfigurationDeclarer.java
@@ -90,9 +90,9 @@
@SuppressWarnings("unchecked")
public T addResource(String resourceName, Number resourceValue) {
- Map<String, Double> resourcesMap =
- (Map<String, Double>) getComponentConfiguration().computeIfAbsent(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP,
- (x) -> new HashMap<String, Double>());
+ Map<String, Double> resourcesMap = (Map<String, Double>) getComponentConfiguration()
+ .computeIfAbsent(Config.TOPOLOGY_COMPONENT_RESOURCES_MAP,
+ (x) -> new HashMap<String, Double>());
resourcesMap.put(resourceName, resourceValue.doubleValue());
return (T) this;
@@ -100,8 +100,6 @@
/**
* Add generic resources for this component.
- *
- * @param resources
*/
@Override
public T addResources(Map<String, Double> resources) {
diff --git a/storm-client/src/jvm/org/apache/storm/topology/BaseStatefulBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/BaseStatefulBoltExecutor.java
index 277cbea..1060d18 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/BaseStatefulBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/BaseStatefulBoltExecutor.java
@@ -12,6 +12,11 @@
package org.apache.storm.topology;
+import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_ACTION;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_TXID;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
+
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -26,11 +31,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_ACTION;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_FIELD_TXID;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
-
/**
* Base class that abstracts the common logic for executing bolts in a stateful topology.
*/
@@ -98,14 +98,14 @@
collector.reportError(th);
}
} else {
- LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, " +
- "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
+ LOG.debug("Waiting for action {}, txid {} from all input tasks. checkPointInputTaskCount {}, "
+ + "transactionRequestCount {}", action, txid, checkPointInputTaskCount, transactionRequestCount);
collector.ack(input);
}
}
/**
- * Checks if check points have been received from all tasks across all input streams to this component
+ * Checks if check points have been received from all tasks across all input streams to this component.
*/
private boolean shouldProcessTransaction(CheckPointState.Action action, long txid) {
TransactionRequest request = new TransactionRequest(action, txid);
@@ -196,10 +196,10 @@
@Override
public String toString() {
- return "TransactionRequest{" +
- "action='" + action + '\'' +
- ", txid=" + txid +
- '}';
+ return "TransactionRequest{"
+ + "action='" + action + '\''
+ + ", txid=" + txid
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/BasicBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/BasicBoltExecutor.java
index a7a3a9e..3a806d0 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/BasicBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/BasicBoltExecutor.java
@@ -22,46 +22,46 @@
public class BasicBoltExecutor implements IRichBolt {
public static final Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class);
- private IBasicBolt _bolt;
- private transient BasicOutputCollector _collector;
+ private IBasicBolt bolt;
+ private transient BasicOutputCollector collector;
public BasicBoltExecutor(IBasicBolt bolt) {
- _bolt = bolt;
+ this.bolt = bolt;
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _bolt.declareOutputFields(declarer);
+ bolt.declareOutputFields(declarer);
}
@Override
public void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector) {
- _bolt.prepare(topoConf, context);
- _collector = new BasicOutputCollector(collector);
+ bolt.prepare(topoConf, context);
+ this.collector = new BasicOutputCollector(collector);
}
@Override
public void execute(Tuple input) {
- _collector.setContext(input);
+ collector.setContext(input);
try {
- _bolt.execute(input, _collector);
- _collector.getOutputter().ack(input);
+ bolt.execute(input, collector);
+ collector.getOutputter().ack(input);
} catch (FailedException e) {
if (e instanceof ReportedFailedException) {
- _collector.reportError(e);
+ collector.reportError(e);
}
- _collector.getOutputter().fail(input);
+ collector.getOutputter().fail(input);
}
}
@Override
public void cleanup() {
- _bolt.cleanup();
+ bolt.cleanup();
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _bolt.getComponentConfiguration();
+ return bolt.getComponentConfiguration();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java b/storm-client/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
index 9ab5f83..d4ed032 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/CheckpointTupleForwarder.java
@@ -12,7 +12,11 @@
package org.apache.storm.topology;
+import static org.apache.storm.spout.CheckPointState.Action;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
+
import java.util.Map;
+
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.tuple.Tuple;
@@ -20,9 +24,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.spout.CheckPointState.Action;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
-
/**
* Wraps {@link IRichBolt} and forwards checkpoint tuples in a stateful topology.
* <p>
diff --git a/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java b/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
index 3915198..081c3ed 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
@@ -62,7 +62,7 @@
protected Config conf = new Config();
- public static void start(ConfigurableTopology topology, String args[]) {
+ public static void start(ConfigurableTopology topology, String[] args) {
String[] remainingArgs = topology.parse(args);
try {
topology.run(remainingArgs);
@@ -78,11 +78,10 @@
new FileInputStream(resource), Charset.defaultCharset()));
if (ret == null) {
ret = new HashMap<>();
- }
- // If the config consists of a single key 'config', its values are used
- // instead. This means that the same config files can be used with Flux
- // and the ConfigurableTopology.
- else {
+ } else {
+ // If the config consists of a single key 'config', its values are used
+ // instead. This means that the same config files can be used with Flux
+ // and the ConfigurableTopology.
if (ret.size() == 1) {
Object confNode = ret.get("config");
if (confNode != null && confNode instanceof Map) {
@@ -98,10 +97,10 @@
return conf;
}
- protected abstract int run(String args[]) throws Exception;
+ protected abstract int run(String[] args) throws Exception;
/**
- * Submits the topology with the name taken from the configuration
+ * Submits the topology with the name taken from the configuration.
**/
protected int submit(Config conf, TopologyBuilder builder) {
String name = (String) Utils.get(conf, Config.TOPOLOGY_NAME, null);
@@ -113,7 +112,7 @@
}
/**
- * Submits the topology under a specific name
+ * Submits the topology under a specific name.
**/
protected int submit(String name, Config conf, TopologyBuilder builder) {
try {
@@ -126,7 +125,7 @@
return 0;
}
- private String[] parse(String args[]) {
+ private String[] parse(String[] args) {
List<String> newArgs = new ArrayList<>();
Collections.addAll(newArgs, args);
diff --git a/storm-client/src/jvm/org/apache/storm/topology/IBasicBolt.java b/storm-client/src/jvm/org/apache/storm/topology/IBasicBolt.java
index 47d620c..12c7a64 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/IBasicBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/IBasicBolt.java
@@ -22,7 +22,7 @@
/**
* Process the input tuple and optionally emit new tuples based on the input tuple.
*
- * All acking is managed for you. Throw a FailedException if you want to fail the tuple.
+ * <p>All acking is managed for you. Throw a FailedException if you want to fail the tuple.
*/
void execute(Tuple input, BasicOutputCollector collector);
diff --git a/storm-client/src/jvm/org/apache/storm/topology/IStatefulBolt.java b/storm-client/src/jvm/org/apache/storm/topology/IStatefulBolt.java
index b3d92f1..5238ad5 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/IStatefulBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/IStatefulBolt.java
@@ -26,16 +26,19 @@
*/
public interface IStatefulBolt<T extends State> extends IStatefulComponent<T> {
/**
+ * Analogue to bolt function.
* @see org.apache.storm.task.IBolt#prepare(Map, TopologyContext, OutputCollector)
*/
void prepare(Map<String, Object> topoConf, TopologyContext context, OutputCollector collector);
/**
+ * Analogue to bolt function.
* @see org.apache.storm.task.IBolt#execute(Tuple)
*/
void execute(Tuple input);
/**
+ * Analogue to bolt function.
* @see org.apache.storm.task.IBolt#cleanup()
*/
void cleanup();
diff --git a/storm-client/src/jvm/org/apache/storm/topology/InputDeclarer.java b/storm-client/src/jvm/org/apache/storm/topology/InputDeclarer.java
index e3badad..92c7c6a 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/InputDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/InputDeclarer.java
@@ -21,135 +21,80 @@
public interface InputDeclarer<T extends InputDeclarer> {
/**
* The stream is partitioned by the fields specified in the grouping.
- *
- * @param componentId
- * @param fields
- * @return
*/
public T fieldsGrouping(String componentId, Fields fields);
/**
* The stream is partitioned by the fields specified in the grouping.
- *
- * @param componentId
- * @param streamId
- * @param fields
- * @return
*/
public T fieldsGrouping(String componentId, String streamId, Fields fields);
/**
* The entire stream goes to a single one of the bolt's tasks. Specifically, it goes to the task with the lowest id.
- *
- * @param componentId
- * @return
*/
public T globalGrouping(String componentId);
/**
* The entire stream goes to a single one of the bolt's tasks. Specifically, it goes to the task with the lowest id.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T globalGrouping(String componentId, String streamId);
/**
* Tuples are randomly distributed across the bolt's tasks in a way such that each bolt is guaranteed to get an equal number of tuples.
- *
- * @param componentId
- * @return
*/
public T shuffleGrouping(String componentId);
/**
* Tuples are randomly distributed across the bolt's tasks in a way such that each bolt is guaranteed to get an equal number of tuples.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T shuffleGrouping(String componentId, String streamId);
/**
* If the target bolt has one or more tasks in the same worker process, tuples will be shuffled to just those in-process tasks.
* Otherwise, this acts like a normal shuffle grouping.
- *
- * @param componentId
- * @return
*/
public T localOrShuffleGrouping(String componentId);
/**
* If the target bolt has one or more tasks in the same worker process, tuples will be shuffled to just those in-process tasks.
* Otherwise, this acts like a normal shuffle grouping.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T localOrShuffleGrouping(String componentId, String streamId);
/**
* This grouping specifies that you don't care how the stream is grouped.
- *
- * @param componentId
- * @return
*/
public T noneGrouping(String componentId);
/**
* This grouping specifies that you don't care how the stream is grouped.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T noneGrouping(String componentId, String streamId);
/**
* The stream is replicated across all the bolt's tasks. Use this grouping with care.
- *
- * @param componentId
- * @return
*/
public T allGrouping(String componentId);
/**
* The stream is replicated across all the bolt's tasks. Use this grouping with care.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T allGrouping(String componentId, String streamId);
/**
* A stream grouped this way means that the producer of the tuple decides which task of the consumer will receive this tuple.
- *
- * @param componentId
- * @return
*/
public T directGrouping(String componentId);
/**
* A stream grouped this way means that the producer of the tuple decides which task of the consumer will receive this tuple.
- *
- * @param componentId
- * @param streamId
- * @return
*/
public T directGrouping(String componentId, String streamId);
/**
* Tuples are passed to two hashing functions and each target task is decided based on the comparison of the state of candidate nodes.
*
- * @param componentId
- * @param fields
- * @return
- *
- * @see https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream
+ * <p>See https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream
* -processing-engines.pdf
*/
public T partialKeyGrouping(String componentId, Fields fields);
@@ -157,32 +102,18 @@
/**
* Tuples are passed to two hashing functions and each target task is decided based on the comparison of the state of candidate nodes.
*
- * @param componentId
- * @param streamId
- * @param fields
- * @return
- *
- * @see https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream
+ * <p>See https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream
* -processing-engines.pdf
*/
public T partialKeyGrouping(String componentId, String streamId, Fields fields);
/**
* A custom stream grouping by implementing the CustomStreamGrouping interface.
- *
- * @param componentId
- * @param grouping
- * @return
*/
public T customGrouping(String componentId, CustomStreamGrouping grouping);
/**
* A custom stream grouping by implementing the CustomStreamGrouping interface.
- *
- * @param componentId
- * @param streamId
- * @param grouping
- * @return
*/
public T customGrouping(String componentId, String streamId, CustomStreamGrouping grouping);
diff --git a/storm-client/src/jvm/org/apache/storm/topology/OutputFieldsGetter.java b/storm-client/src/jvm/org/apache/storm/topology/OutputFieldsGetter.java
index 1cd268c..3d4fddf 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/OutputFieldsGetter.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/OutputFieldsGetter.java
@@ -19,7 +19,7 @@
import org.apache.storm.utils.Utils;
public class OutputFieldsGetter implements OutputFieldsDeclarer {
- private Map<String, StreamInfo> _fields = new HashMap<>();
+ private Map<String, StreamInfo> fields = new HashMap<>();
@Override
public void declare(Fields fields) {
@@ -41,15 +41,15 @@
if (null == streamId) {
throw new IllegalArgumentException("streamId can't be null");
}
- if (_fields.containsKey(streamId)) {
+ if (this.fields.containsKey(streamId)) {
throw new IllegalArgumentException("Fields for " + streamId + " already set");
}
- _fields.put(streamId, new StreamInfo(fields.toList(), direct));
+ this.fields.put(streamId, new StreamInfo(fields.toList(), direct));
}
public Map<String, StreamInfo> getFieldsDeclaration() {
- return _fields;
+ return fields;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java
index cc33ec1..f70db21 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/PersistentWindowedBoltExecutor.java
@@ -12,6 +12,8 @@
package org.apache.storm.topology;
+import static org.apache.storm.windowing.persistence.WindowState.WindowPartition;
+
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashMap;
@@ -38,8 +40,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.windowing.persistence.WindowState.WindowPartition;
-
/**
* Wraps a {@link IStatefulWindowedBolt} and handles the execution. Uses state and the underlying checkpointing mechanisms to save the
* tuples in window to state. The tuples are also kept in-memory by transparently caching the window partitions and checkpointing them as
diff --git a/storm-client/src/jvm/org/apache/storm/topology/ResourceDeclarer.java b/storm-client/src/jvm/org/apache/storm/topology/ResourceDeclarer.java
index 8d04ddc..8cba971 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/ResourceDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/ResourceDeclarer.java
@@ -20,7 +20,7 @@
*/
public interface ResourceDeclarer<T extends ResourceDeclarer> {
/**
- * Set the amount of on heap memory for this component
+ * Set the amount of on heap memory for this component.
*
* @param onHeap the amount of on heap memory
* @return this for chaining
@@ -28,7 +28,7 @@
T setMemoryLoad(Number onHeap);
/**
- * Set the amount of memory for this component on and off heap
+ * Set the amount of memory for this component on and off heap.
*
* @param onHeap the amount of on heap memory
* @param offHeap the amount of off heap memory
@@ -37,11 +37,12 @@
T setMemoryLoad(Number onHeap, Number offHeap);
/**
- * Set the amount of CPU load for this component
+ * Set the amount of CPU load for this component.
*
* @param amount the amount of CPU
* @return this for chaining
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
T setCPULoad(Number amount);
/**
diff --git a/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinNode.java b/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinNode.java
index 0875196..0803cb1 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinNode.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinNode.java
@@ -16,13 +16,13 @@
import org.apache.storm.utils.Utils;
/**
- * A request for a shared memory region off heap between workers on a node
+ * A request for a shared memory region off heap between workers on a node.
*/
public class SharedOffHeapWithinNode extends SharedMemory {
private static final long serialVersionUID = 1L;
/**
- * Create a new request for a shared memory region off heap between workers on a node
+ * Create a new request for a shared memory region off heap between workers on a node.
*
* @param amount the number of MB to share
* @param name the name of the shared region (for tracking purposes)
diff --git a/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinWorker.java b/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinWorker.java
index b04f567..7d2aacb 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinWorker.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/SharedOffHeapWithinWorker.java
@@ -20,7 +20,7 @@
*/
public class SharedOffHeapWithinWorker extends SharedMemory {
/**
- * Create a new request for a shared memory region off heap within a worker
+ * Create a new request for a shared memory region off heap within a worker.
*
* @param amount the number of MB to share
* @param name the name of the shared region (for tracking purposes)
diff --git a/storm-client/src/jvm/org/apache/storm/topology/SharedOnHeap.java b/storm-client/src/jvm/org/apache/storm/topology/SharedOnHeap.java
index 4aa7681..b678c31 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/SharedOnHeap.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/SharedOnHeap.java
@@ -22,7 +22,7 @@
private static final long serialVersionUID = 1L;
/**
- * Create a new request for a shared memory region on heap
+ * Create a new request for a shared memory region on heap.
*
* @param amount the number of MB to share on heap
* @param name the name of the shared region (for tracking purposes)
diff --git a/storm-client/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
index 280a713..d2202c4 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/StatefulBoltExecutor.java
@@ -12,6 +12,12 @@
package org.apache.storm.topology;
+import static org.apache.storm.spout.CheckPointState.Action;
+import static org.apache.storm.spout.CheckPointState.Action.COMMIT;
+import static org.apache.storm.spout.CheckPointState.Action.INITSTATE;
+import static org.apache.storm.spout.CheckPointState.Action.PREPARE;
+import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
+
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -28,12 +34,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.spout.CheckPointState.Action;
-import static org.apache.storm.spout.CheckPointState.Action.COMMIT;
-import static org.apache.storm.spout.CheckPointState.Action.INITSTATE;
-import static org.apache.storm.spout.CheckPointState.Action.PREPARE;
-import static org.apache.storm.spout.CheckPointState.Action.ROLLBACK;
-
/**
* Wraps a {@link IStatefulBolt} and manages the state of the bolt.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
index 2bd9981..0bde3e7 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java
@@ -112,8 +112,11 @@
} else if (msgId <= state.lastEvaluated) {
super.execute(input);
} else {
- LOG.debug("Tuple msg id {} > lastEvaluated id {}, adding to pendingTuples and clearing recovery state " +
- "for taskStream {}", msgId, state.lastEvaluated, taskStream);
+ LOG.debug("Tuple msg id {} > lastEvaluated id {}, adding to pendingTuples and clearing recovery state "
+ + "for taskStream {}",
+ msgId,
+ state.lastEvaluated,
+ taskStream);
pendingTuples.add(input);
clearRecoveryState(taskStream);
}
@@ -203,8 +206,12 @@
@Override
public void onActivation(List<Tuple> events, List<Tuple> newEvents, List<Tuple> expired, Long timestamp) {
if (isRecovering()) {
- String msg = String.format("Unexpected activation with events %s, newEvents %s, expired %s in recovering state. " +
- "recoveryStates %s ", events, newEvents, expired, recoveryStates);
+ String msg = String.format("Unexpected activation with events %s, newEvents %s, expired %s in "
+ + "recovering state. recoveryStates %s ",
+ events,
+ newEvents,
+ expired,
+ recoveryStates);
LOG.error(msg);
throw new IllegalStateException(msg);
} else {
@@ -318,10 +325,10 @@
@Override
public String toString() {
- return "WindowState{" +
- "lastExpired=" + lastExpired +
- ", lastEvaluated=" + lastEvaluated +
- '}';
+ return "WindowState{"
+ + "lastExpired=" + lastExpired
+ + ", lastEvaluated=" + lastEvaluated
+ + '}';
}
}
@@ -369,10 +376,10 @@
@Override
public String toString() {
- return "TaskStream{" +
- "sourceTask=" + sourceTask +
- ", streamId=" + streamId +
- '}';
+ return "TaskStream{"
+ + "sourceTask=" + sourceTask
+ + ", streamId=" + streamId
+ + '}';
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java
index eafc728..63384df 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java
@@ -12,6 +12,10 @@
package org.apache.storm.topology;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_COMPONENT_ID;
+import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
+import static org.apache.storm.utils.Utils.parseJson;
+
import java.io.NotSerializableException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -50,52 +54,48 @@
import org.apache.storm.utils.Utils;
import org.apache.storm.windowing.TupleWindow;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_COMPONENT_ID;
-import static org.apache.storm.spout.CheckpointSpout.CHECKPOINT_STREAM_ID;
-import static org.apache.storm.utils.Utils.parseJson;
-
/**
* TopologyBuilder exposes the Java API for specifying a topology for Storm to execute. Topologies are Thrift structures in the end, but
* since the Thrift API is so verbose, TopologyBuilder greatly eases the process of creating topologies. The template for creating and
* submitting a topology looks something like:
*
- * ```java TopologyBuilder builder = new TopologyBuilder();
+ * <p>```java TopologyBuilder builder = new TopologyBuilder();
*
- * builder.setSpout("1", new TestWordSpout(true), 5); builder.setSpout("2", new TestWordSpout(true), 3); builder.setBolt("3", new
+ * <p>builder.setSpout("1", new TestWordSpout(true), 5); builder.setSpout("2", new TestWordSpout(true), 3); builder.setBolt("3", new
* TestWordCounter(), 3) .fieldsGrouping("1", new Fields("word")) .fieldsGrouping("2", new Fields("word")); builder.setBolt("4", new
* TestGlobalCount()) .globalGrouping("1");
*
- * Map<String, Object> conf = new HashMap(); conf.put(Config.TOPOLOGY_WORKERS, 4);
+ * <p>Map<String, Object> conf = new HashMap(); conf.put(Config.TOPOLOGY_WORKERS, 4);
*
- * StormSubmitter.submitTopology("mytopology", conf, builder.createTopology()); ```
+ * <p>StormSubmitter.submitTopology("mytopology", conf, builder.createTopology()); ```
*
- * Running the exact same topology in local mode (in process), and configuring it to log all tuples emitted, looks like the following. Note
- * that it lets the topology run for 10 seconds before shutting down the local cluster.
+ * <p>Running the exact same topology in local mode (in process), and configuring it to log all tuples emitted, looks
+ * like the following. Note that it lets the topology run for 10 seconds before shutting down the local cluster.
*
- * ```java TopologyBuilder builder = new TopologyBuilder();
+ * <p>```java TopologyBuilder builder = new TopologyBuilder();
*
- * builder.setSpout("1", new TestWordSpout(true), 5); builder.setSpout("2", new TestWordSpout(true), 3); builder.setBolt("3", new
+ * <p>builder.setSpout("1", new TestWordSpout(true), 5); builder.setSpout("2", new TestWordSpout(true), 3); builder.setBolt("3", new
* TestWordCounter(), 3) .fieldsGrouping("1", new Fields("word")) .fieldsGrouping("2", new Fields("word")); builder.setBolt("4", new
* TestGlobalCount()) .globalGrouping("1");
*
- * Map<String, Object> conf = new HashMap(); conf.put(Config.TOPOLOGY_WORKERS, 4); conf.put(Config.TOPOLOGY_DEBUG, true);
+ * <p>Map<String, Object> conf = new HashMap(); conf.put(Config.TOPOLOGY_WORKERS, 4); conf.put(Config.TOPOLOGY_DEBUG, true);
*
- * try (LocalCluster cluster = new LocalCluster(); LocalTopology topo = cluster.submitTopology("mytopology", conf,
+ * <p>try (LocalCluster cluster = new LocalCluster(); LocalTopology topo = cluster.submitTopology("mytopology", conf,
* builder.createTopology());){ Utils.sleep(10000); } ```
*
- * The pattern for `TopologyBuilder` is to map component ids to components using the setSpout and setBolt methods. Those methods return
+ * <p>The pattern for `TopologyBuilder` is to map component ids to components using the setSpout and setBolt methods. Those methods return
* objects that are then used to declare the inputs for that component.
*/
public class TopologyBuilder {
- private final Map<String, IRichBolt> _bolts = new HashMap<>();
- private final Map<String, IRichSpout> _spouts = new HashMap<>();
+ private final Map<String, IRichBolt> bolts = new HashMap<>();
+ private final Map<String, IRichSpout> spouts = new HashMap<>();
private final Map<String, ComponentCommon> commons = new HashMap<>();
- private final Map<String, Set<String>> _componentToSharedMemory = new HashMap<>();
- private final Map<String, SharedMemory> _sharedMemory = new HashMap<>();
+ private final Map<String, Set<String>> componentToSharedMemory = new HashMap<>();
+ private final Map<String, SharedMemory> sharedMemory = new HashMap<>();
private boolean hasStatefulBolt = false;
- private Map<String, StateSpoutSpec> _stateSpouts = new HashMap<>();
- private List<ByteBuffer> _workerHooks = new ArrayList<>();
+ private Map<String, StateSpoutSpec> stateSpouts = new HashMap<>();
+ private List<ByteBuffer> workerHooks = new ArrayList<>();
private static String mergeIntoJson(Map<String, Object> into, Map<String, Object> newMap) {
Map<String, Object> res = new HashMap<>(into);
@@ -107,8 +107,8 @@
Map<String, Bolt> boltSpecs = new HashMap<>();
Map<String, SpoutSpec> spoutSpecs = new HashMap<>();
maybeAddCheckpointSpout();
- for (String boltId : _bolts.keySet()) {
- IRichBolt bolt = _bolts.get(boltId);
+ for (String boltId : bolts.keySet()) {
+ IRichBolt bolt = bolts.get(boltId);
bolt = maybeAddCheckpointTupleForwarder(bolt);
ComponentCommon common = getComponentCommon(boltId, bolt);
try {
@@ -116,25 +116,35 @@
boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.javaSerialize(bolt)), common));
} catch (RuntimeException wrapperCause) {
if (wrapperCause.getCause() != null && NotSerializableException.class.equals(wrapperCause.getCause().getClass())) {
- throw new IllegalStateException(
- "Bolt '" + boltId + "' contains a non-serializable field of type " + wrapperCause.getCause().getMessage() + ", " +
- "which was instantiated prior to topology creation. " + wrapperCause.getCause().getMessage() + " " +
- "should be instantiated within the prepare method of '" + boltId + " at the earliest.", wrapperCause);
+ throw new IllegalStateException("Bolt '" + boltId + "' contains a non-serializable field of type "
+ + wrapperCause.getCause().getMessage() + ", "
+ + "which was instantiated prior to topology creation. "
+ + wrapperCause.getCause().getMessage()
+ + " "
+ + "should be instantiated within the prepare method of '"
+ + boltId
+ + " at the earliest.",
+ wrapperCause);
}
throw wrapperCause;
}
}
- for (String spoutId : _spouts.keySet()) {
- IRichSpout spout = _spouts.get(spoutId);
+ for (String spoutId : spouts.keySet()) {
+ IRichSpout spout = spouts.get(spoutId);
ComponentCommon common = getComponentCommon(spoutId, spout);
try {
spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.javaSerialize(spout)), common));
} catch (RuntimeException wrapperCause) {
if (wrapperCause.getCause() != null && NotSerializableException.class.equals(wrapperCause.getCause().getClass())) {
throw new IllegalStateException(
- "Spout '" + spoutId + "' contains a non-serializable field of type " + wrapperCause.getCause().getMessage() + ", " +
- "which was instantiated prior to topology creation. " + wrapperCause.getCause().getMessage() + " " +
- "should be instantiated within the open method of '" + spoutId + " at the earliest.", wrapperCause);
+ "Spout '" + spoutId + "' contains a non-serializable field of type "
+ + wrapperCause.getCause().getMessage()
+ + ", which was instantiated prior to topology creation. "
+ + wrapperCause.getCause().getMessage()
+ + " should be instantiated within the open method of '"
+ + spoutId
+ + " at the earliest.",
+ wrapperCause);
}
throw wrapperCause;
}
@@ -144,11 +154,11 @@
boltSpecs,
new HashMap<>());
- stormTopology.set_worker_hooks(_workerHooks);
+ stormTopology.set_worker_hooks(workerHooks);
- if (!_componentToSharedMemory.isEmpty()) {
- stormTopology.set_component_to_shared_memory(_componentToSharedMemory);
- stormTopology.set_shared_memory(_sharedMemory);
+ if (!componentToSharedMemory.isEmpty()) {
+ stormTopology.set_component_to_shared_memory(componentToSharedMemory);
+ stormTopology.set_shared_memory(sharedMemory);
}
return Utils.addVersions(stormTopology);
@@ -173,16 +183,16 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param bolt the bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somewhere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism_hint) throws IllegalArgumentException {
+ public BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelismHint) throws IllegalArgumentException {
validateUnusedId(id);
- initCommon(id, bolt, parallelism_hint);
- _bolts.put(id, bolt);
+ initCommon(id, bolt, parallelismHint);
+ bolts.put(id, bolt);
return new BoltGetter(id);
}
@@ -209,14 +219,14 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param bolt the basic bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somewhere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism_hint) throws IllegalArgumentException {
- return setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint);
+ public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelismHint) throws IllegalArgumentException {
+ return setBolt(id, new BasicBoltExecutor(bolt), parallelismHint);
}
/**
@@ -240,14 +250,14 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param bolt the windowed bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somwehere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public BoltDeclarer setBolt(String id, IWindowedBolt bolt, Number parallelism_hint) throws IllegalArgumentException {
- return setBolt(id, new WindowedBoltExecutor(bolt), parallelism_hint);
+ public BoltDeclarer setBolt(String id, IWindowedBolt bolt, Number parallelismHint) throws IllegalArgumentException {
+ return setBolt(id, new WindowedBoltExecutor(bolt), parallelismHint);
}
/**
@@ -281,16 +291,16 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param bolt the stateful bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somwehere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public <T extends State> BoltDeclarer setBolt(String id, IStatefulBolt<T> bolt, Number parallelism_hint) throws
+ public <T extends State> BoltDeclarer setBolt(String id, IStatefulBolt<T> bolt, Number parallelismHint) throws
IllegalArgumentException {
hasStatefulBolt = true;
- return setBolt(id, new StatefulBoltExecutor<T>(bolt), parallelism_hint);
+ return setBolt(id, new StatefulBoltExecutor<T>(bolt), parallelismHint);
}
/**
@@ -319,14 +329,14 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param bolt the stateful windowed bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somwehere around the cluster.
* @param <T> the type of the state (e.g. {@link org.apache.storm.state.KeyValueState})
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public <T extends State> BoltDeclarer setBolt(String id, IStatefulWindowedBolt<T> bolt, Number parallelism_hint) throws
+ public <T extends State> BoltDeclarer setBolt(String id, IStatefulWindowedBolt<T> bolt, Number parallelismHint) throws
IllegalArgumentException {
hasStatefulBolt = true;
IStatefulBolt<T> executor;
@@ -335,7 +345,7 @@
} else {
executor = new StatefulWindowedBoltExecutor<T>(bolt);
}
- return setBolt(id, new StatefulBoltExecutor<T>(executor), parallelism_hint);
+ return setBolt(id, new StatefulBoltExecutor<T>(executor), parallelismHint);
}
/**
@@ -364,15 +374,15 @@
* outputs.
* @param biConsumer lambda expression that implements tuple processing for this bolt
* @param fields fields for tuple that should be emitted to downstream bolts
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somewhere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public BoltDeclarer setBolt(String id, SerializableBiConsumer<Tuple, BasicOutputCollector> biConsumer, Number parallelism_hint,
+ public BoltDeclarer setBolt(String id, SerializableBiConsumer<Tuple, BasicOutputCollector> biConsumer, Number parallelismHint,
String... fields) throws IllegalArgumentException {
- return setBolt(id, new LambdaBiConsumerBolt(biConsumer, fields), parallelism_hint);
+ return setBolt(id, new LambdaBiConsumerBolt(biConsumer, fields), parallelismHint);
}
/**
@@ -398,14 +408,14 @@
* @param id the id of this component. This id is referenced by other components that want to consume this bolt's
* outputs.
* @param consumer lambda expression that implements tuple processing for this bolt
- * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
+ * @param parallelismHint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process
* somewhere around the cluster.
* @return use the returned object to declare the inputs to this component
*
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public BoltDeclarer setBolt(String id, SerializableConsumer<Tuple> consumer, Number parallelism_hint) throws IllegalArgumentException {
- return setBolt(id, new LambdaConsumerBolt(consumer), parallelism_hint);
+ public BoltDeclarer setBolt(String id, SerializableConsumer<Tuple> consumer, Number parallelismHint) throws IllegalArgumentException {
+ return setBolt(id, new LambdaConsumerBolt(consumer), parallelismHint);
}
/**
@@ -425,27 +435,18 @@
*
* @param id the id of this component. This id is referenced by other components that want to consume this spout's
* outputs.
- * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
+ * @param parallelismHint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
* process somewhere around the cluster.
* @param spout the spout
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public SpoutDeclarer setSpout(String id, IRichSpout spout, Number parallelism_hint) throws IllegalArgumentException {
+ public SpoutDeclarer setSpout(String id, IRichSpout spout, Number parallelismHint) throws IllegalArgumentException {
validateUnusedId(id);
- initCommon(id, spout, parallelism_hint);
- _spouts.put(id, spout);
+ initCommon(id, spout, parallelismHint);
+ spouts.put(id, spout);
return new SpoutGetter(id);
}
- public void setStateSpout(String id, IRichStateSpout stateSpout) throws IllegalArgumentException {
- setStateSpout(id, stateSpout, null);
- }
-
- public void setStateSpout(String id, IRichStateSpout stateSpout, Number parallelism_hint) throws IllegalArgumentException {
- validateUnusedId(id);
- // TODO: finish
- }
-
/**
* Define a new spout in this topology.
*
@@ -463,17 +464,26 @@
*
* @param id the id of this component. This id is referenced by other components that want to consume this spout's
* outputs.
- * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
+ * @param parallelismHint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
* process somewhere around the cluster.
* @param supplier lambda expression that implements tuple generating for this spout
* @throws IllegalArgumentException if {@code parallelism_hint} is not positive
*/
- public SpoutDeclarer setSpout(String id, SerializableSupplier<?> supplier, Number parallelism_hint) throws IllegalArgumentException {
- return setSpout(id, new LambdaSpout(supplier), parallelism_hint);
+ public SpoutDeclarer setSpout(String id, SerializableSupplier<?> supplier, Number parallelismHint) throws IllegalArgumentException {
+ return setSpout(id, new LambdaSpout(supplier), parallelismHint);
+ }
+
+ public void setStateSpout(String id, IRichStateSpout stateSpout) throws IllegalArgumentException {
+ setStateSpout(id, stateSpout, null);
+ }
+
+ public void setStateSpout(String id, IRichStateSpout stateSpout, Number parallelismHint) throws IllegalArgumentException {
+ validateUnusedId(id);
+ // TODO: finish
}
/**
- * Add a new worker lifecycle hook
+ * Add a new worker lifecycle hook.
*
* @param workerHook the lifecycle hook to add
*/
@@ -482,17 +492,17 @@
throw new IllegalArgumentException("WorkerHook must not be null.");
}
- _workerHooks.add(ByteBuffer.wrap(Utils.javaSerialize(workerHook)));
+ workerHooks.add(ByteBuffer.wrap(Utils.javaSerialize(workerHook)));
}
private void validateUnusedId(String id) {
- if (_bolts.containsKey(id)) {
+ if (bolts.containsKey(id)) {
throw new IllegalArgumentException("Bolt has already been declared for id " + id);
}
- if (_spouts.containsKey(id)) {
+ if (spouts.containsKey(id)) {
throw new IllegalArgumentException("Spout has already been declared for id " + id);
}
- if (_stateSpouts.containsKey(id)) {
+ if (stateSpouts.containsKey(id)) {
throw new IllegalArgumentException("State spout has already been declared for id " + id);
}
}
@@ -531,7 +541,7 @@
Set<GlobalStreamId> checkPointInputs = new HashSet<>();
for (GlobalStreamId inputStream : component.get_inputs().keySet()) {
String sourceId = inputStream.get_componentId();
- if (_spouts.containsKey(sourceId)) {
+ if (spouts.containsKey(sourceId)) {
checkPointInputs.add(new GlobalStreamId(CHECKPOINT_COMPONENT_ID, CHECKPOINT_STREAM_ID));
} else {
checkPointInputs.add(new GlobalStreamId(sourceId, CHECKPOINT_STREAM_ID));
@@ -627,12 +637,12 @@
@SuppressWarnings("unchecked")
@Override
public T addSharedMemory(SharedMemory request) {
- SharedMemory found = _sharedMemory.get(request.get_name());
+ SharedMemory found = sharedMemory.get(request.get_name());
if (found != null && !found.equals(request)) {
throw new IllegalArgumentException("Cannot have multiple different shared memory regions with the same name");
}
- _sharedMemory.put(request.get_name(), request);
- Set<String> mems = _componentToSharedMemory.computeIfAbsent(id, (k) -> new HashSet<>());
+ sharedMemory.put(request.get_name(), request);
+ Set<String> mems = componentToSharedMemory.computeIfAbsent(id, (k) -> new HashSet<>());
mems.add(request.get_name());
return (T) this;
}
@@ -645,11 +655,11 @@
}
protected class BoltGetter extends ConfigGetter<BoltDeclarer> implements BoltDeclarer {
- private String _boltId;
+ private String boltId;
public BoltGetter(String boltId) {
super(boltId);
- _boltId = boltId;
+ this.boltId = boltId;
}
public BoltDeclarer fieldsGrouping(String componentId, Fields fields) {
@@ -709,11 +719,16 @@
}
private BoltDeclarer grouping(String componentId, String streamId, Grouping grouping) {
- commons.get(_boltId).put_to_inputs(new GlobalStreamId(componentId, streamId), grouping);
+ commons.get(boltId).put_to_inputs(new GlobalStreamId(componentId, streamId), grouping);
return this;
}
@Override
+ public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) {
+ return grouping(id.get_componentId(), id.get_streamId(), grouping);
+ }
+
+ @Override
public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) {
return customGrouping(componentId, new PartialKeyGrouping(fields));
}
@@ -732,10 +747,5 @@
public BoltDeclarer customGrouping(String componentId, String streamId, CustomStreamGrouping grouping) {
return grouping(componentId, streamId, Grouping.custom_serialized(Utils.javaSerialize(grouping)));
}
-
- @Override
- public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) {
- return grouping(id.get_componentId(), id.get_streamId(), grouping);
- }
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/TupleFieldTimestampExtractor.java b/storm-client/src/jvm/org/apache/storm/topology/TupleFieldTimestampExtractor.java
index efb5948..ac5cb7f 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/TupleFieldTimestampExtractor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/TupleFieldTimestampExtractor.java
@@ -36,8 +36,8 @@
@Override
public String toString() {
- return "TupleFieldTimestampExtractor{" +
- "fieldName='" + fieldName + '\'' +
- '}';
+ return "TupleFieldTimestampExtractor{"
+ + "fieldName='" + fieldName + '\''
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
index e923622..e6a12d9 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
@@ -12,6 +12,9 @@
package org.apache.storm.topology;
+import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
@@ -53,9 +56,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
-import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
-
/**
* An {@link IWindowedBolt} wrapper that does the windowing of tuples.
*/
@@ -106,17 +106,16 @@
private void ensureDurationLessThanTimeout(int duration, int timeout) {
if (duration > timeout) {
- throw new IllegalArgumentException("Window duration (length + sliding interval) value " + duration +
- " is more than " + Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS +
- " value " + timeout);
+ throw new IllegalArgumentException("Window duration (length + sliding interval) value " + duration
+ + " is more than " + Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS + " value " + timeout);
}
}
private void ensureCountLessThanMaxPending(int count, int maxPending) {
if (count > maxPending) {
- throw new IllegalArgumentException("Window count (length + sliding interval) value " + count +
- " is more than " + Config.TOPOLOGY_MAX_SPOUT_PENDING +
- " value " + maxPending);
+ throw new IllegalArgumentException("Window count (length + sliding interval) value " + count
+ + " is more than " + Config.TOPOLOGY_MAX_SPOUT_PENDING
+ + " value " + maxPending);
}
}
@@ -149,9 +148,9 @@
private WindowManager<Tuple> initWindowManager(WindowLifecycleListener<Tuple> lifecycleListener, Map<String, Object> topoConf,
TopologyContext context, Collection<Event<Tuple>> queue, boolean stateful) {
- WindowManager<Tuple> manager = stateful ?
- new StatefulWindowManager<>(lifecycleListener, queue)
- : new WindowManager<>(lifecycleListener, queue);
+ WindowManager<Tuple> manager = stateful
+ ? new StatefulWindowManager<>(lifecycleListener, queue)
+ : new WindowManager<>(lifecycleListener, queue);
Count windowLengthCount = null;
Duration slidingIntervalDuration = null;
@@ -234,7 +233,7 @@
}
/**
- * Start the trigger policy and waterMarkEventGenerator if set
+ * Start the trigger policy and waterMarkEventGenerator if set.
*/
protected void start() {
if (waterMarkEventGenerator != null) {
diff --git a/storm-client/src/jvm/org/apache/storm/topology/base/BaseTickTupleAwareRichBolt.java b/storm-client/src/jvm/org/apache/storm/topology/base/BaseTickTupleAwareRichBolt.java
index 4df2702..d901654 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/base/BaseTickTupleAwareRichBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/base/BaseTickTupleAwareRichBolt.java
@@ -43,7 +43,7 @@
* Process a single tick tuple of input. Tick tuple doesn't need to be acked. It provides default "DO NOTHING" implementation for
* convenient. Override this method if needed.
*
- * More details on {@link org.apache.storm.task.IBolt#execute(Tuple)}.
+ * <p>More details on {@link org.apache.storm.task.IBolt#execute(Tuple)}.
*
* @param tuple The input tuple to be processed.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java b/storm-client/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
index 937841b..f411e87 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java
@@ -289,9 +289,9 @@
@Override
public String toString() {
- return "Count{" +
- "value=" + value +
- '}';
+ return "Count{"
+ + "value=" + value
+ + '}';
}
}
@@ -384,9 +384,9 @@
@Override
public String toString() {
- return "Duration{" +
- "value=" + value +
- '}';
+ return "Duration{"
+ + "value=" + value
+ + '}';
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/transactional/TransactionAttempt.java b/storm-client/src/jvm/org/apache/storm/transactional/TransactionAttempt.java
index 683e414..150c930 100644
--- a/storm-client/src/jvm/org/apache/storm/transactional/TransactionAttempt.java
+++ b/storm-client/src/jvm/org/apache/storm/transactional/TransactionAttempt.java
@@ -19,8 +19,8 @@
*/
@Deprecated
public class TransactionAttempt {
- BigInteger _txid;
- long _attemptId;
+ BigInteger txid;
+ long attemptId;
// for kryo compatibility
@@ -29,21 +29,21 @@
}
public TransactionAttempt(BigInteger txid, long attemptId) {
- _txid = txid;
- _attemptId = attemptId;
+ this.txid = txid;
+ this.attemptId = attemptId;
}
public BigInteger getTransactionId() {
- return _txid;
+ return txid;
}
public long getAttemptId() {
- return _attemptId;
+ return attemptId;
}
@Override
public int hashCode() {
- return _txid.hashCode();
+ return txid.hashCode();
}
@Override
@@ -52,11 +52,11 @@
return false;
}
TransactionAttempt other = (TransactionAttempt) o;
- return _txid.equals(other._txid) && _attemptId == other._attemptId;
+ return txid.equals(other.txid) && attemptId == other.attemptId;
}
@Override
public String toString() {
- return "" + _txid + ":" + _attemptId;
+ return "" + txid + ":" + attemptId;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/JoinOutFieldsMode.java b/storm-client/src/jvm/org/apache/storm/trident/JoinOutFieldsMode.java
index 60ce0fd..d095b6e 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/JoinOutFieldsMode.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/JoinOutFieldsMode.java
@@ -15,12 +15,12 @@
/**
* This enum defines how the output fields of JOIN is constructed.
*
- * If user specifies COMPACT while calling JOIN, the tuples emitted from the join will contain: First, the list of join fields. Please note
- * that joining fields are exposed only once from emitted tuples. Next, a list of all non-join fields from all streams, in order of how the
- * streams were passed to the join method.
+ * <p>If user specifies COMPACT while calling JOIN, the tuples emitted from the join will contain: First, the list of
+ * join fields. Please note that joining fields are exposed only once from emitted tuples. Next, a list of all non-join
+ * fields from all streams, in order of how the streams were passed to the join method.
*
- * If user specifies PRESERVE while calling JOIN, the tuples emitted from the join will contain: a list of all fields from all streams, in
- * order of how the streams were passed to the join method.
+ * <p>If user specifies PRESERVE while calling JOIN, the tuples emitted from the join will contain: a list of all fields
+ * from all streams, in order of how the streams were passed to the join method.
*/
public enum JoinOutFieldsMode {
COMPACT,
diff --git a/storm-client/src/jvm/org/apache/storm/trident/Stream.java b/storm-client/src/jvm/org/apache/storm/trident/Stream.java
index 6e77e0e..3695d46 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/Stream.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/Stream.java
@@ -86,9 +86,9 @@
* batches. A stream is partitioned accross the nodes in the cluster, and operations are applied to a stream in parallel accross each
* partition.
*
- * There are five types of operations that can be performed on streams in Trident
+ * <p>There are five types of operations that can be performed on streams in Trident
*
- * 1. **Partiton-Local Operations** - Operations that are applied locally to each partition and do not involve network transfer 2.
+ * <p>1. **Partiton-Local Operations** - Operations that are applied locally to each partition and do not involve network transfer 2.
* **Repartitioning Operations** - Operations that change how tuples are partitioned across tasks(thus causing network transfer), but do not
* change the content of the stream. 3. **Aggregation Operations** - Operations that *may* repartition a stream (thus causing network
* transfer) 4. **Grouping Operations** - Operations that may repartition a stream on specific fields and group together tuples whose fields
@@ -96,14 +96,14 @@
*/
// TODO: need to be able to replace existing fields with the function fields (like Cascading Fields.REPLACE)
public class Stream implements IAggregatableStream, ResourceDeclarer<Stream> {
- final Node _node;
- final String _name;
- private final TridentTopology _topology;
+ final Node node;
+ final String name;
+ private final TridentTopology topology;
protected Stream(TridentTopology topology, String name, Node node) {
- _topology = topology;
- _node = node;
- _name = name;
+ this.topology = topology;
+ this.node = node;
+ this.name = name;
}
/**
@@ -111,38 +111,34 @@
* visible in the Storm UI.
*
* @param name - The label to apply to the stream
- * @return
*/
public Stream name(String name) {
- return new Stream(_topology, name, _node);
+ return new Stream(topology, name, node);
}
/**
* Applies a parallelism hint to a stream.
- *
- * @param hint
- * @return
*/
public Stream parallelismHint(int hint) {
- _node.parallelismHint = hint;
+ node.parallelismHint = hint;
return this;
}
/**
- * Sets the CPU Load resource for the current operation
+ * Sets the CPU Load resource for the current operation.
*/
@Override
public Stream setCPULoad(Number load) {
- _node.setCPULoad(load);
+ node.setCPULoad(load);
return this;
}
/**
- * Sets the Memory Load resources for the current operation. offHeap becomes default
+ * Sets the Memory Load resources for the current operation. offHeap becomes default.
*/
@Override
public Stream setMemoryLoad(Number onHeap) {
- _node.setMemoryLoad(onHeap);
+ node.setMemoryLoad(onHeap);
return this;
}
@@ -151,39 +147,39 @@
*/
@Override
public Stream setMemoryLoad(Number onHeap, Number offHeap) {
- _node.setMemoryLoad(onHeap, offHeap);
+ node.setMemoryLoad(onHeap, offHeap);
return this;
}
@Override
public Stream addSharedMemory(SharedMemory request) {
- _node.addSharedMemory(request);
+ node.addSharedMemory(request);
return this;
}
/**
* Filters out fields from a stream, resulting in a Stream containing only the fields specified by `keepFields`.
*
- * For example, if you had a Stream `mystream` containing the fields `["a", "b", "c","d"]`, calling"
+ * <p>For example, if you had a Stream `mystream` containing the fields `["a", "b", "c","d"]`, calling"
*
- * ```java mystream.project(new Fields("b", "d")) ```
+ * <p>```java mystream.project(new Fields("b", "d")) ```
*
- * would produce a stream containing only the fields `["b", "d"]`.
+ * <p>would produce a stream containing only the fields `["b", "d"]`.
*
* @param keepFields The fields in the Stream to keep
- * @return
*/
public Stream project(Fields keepFields) {
projectionValidation(keepFields);
- return _topology.addSourcedNode(this, new ProcessorNode(_topology.getUniqueStreamId(), _name, keepFields, new Fields(),
- new ProjectedProcessor(keepFields)));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ keepFields,
+ new Fields(),
+ new ProjectedProcessor(keepFields)));
}
/**
- * ## Grouping Operation
- *
- * @param fields
- * @return
+ * ## Grouping Operation.
*/
public GroupedStream groupBy(Fields fields) {
projectionValidation(fields);
@@ -191,10 +187,7 @@
}
/**
- * ## Repartitioning Operation
- *
- * @param fields
- * @return
+ * ## Repartitioning Operation.
*/
public Stream partitionBy(Fields fields) {
projectionValidation(fields);
@@ -202,32 +195,38 @@
}
/**
- * ## Repartitioning Operation
- *
- * @param partitioner
- * @return
+ * ## Repartitioning Operation.
*/
public Stream partition(CustomStreamGrouping partitioner) {
return partition(Grouping.custom_serialized(Utils.javaSerialize(partitioner)));
}
/**
- * ## Repartitioning Operation
+ * ## Repartitioning Operation.
*
- * Use random round robin algorithm to evenly redistribute tuples across all target partitions
+ * <p>This method takes in a custom partitioning function that implements {@link org.apache.storm.grouping.CustomStreamGrouping}
+ */
+ public Stream partition(Grouping grouping) {
+ if (node instanceof PartitionNode) {
+ return each(new Fields(), new TrueFilter()).partition(grouping);
+ } else {
+ return topology.addSourcedNode(this, new PartitionNode(node.streamId, name, getOutputFields(), grouping));
+ }
+ }
+
+ /**
+ * ## Repartitioning Operation.
*
- * @return
+ * <p>Use random round robin algorithm to evenly redistribute tuples across all target partitions.
*/
public Stream shuffle() {
return partition(Grouping.shuffle(new NullStruct()));
}
/**
- * ## Repartitioning Operation
+ * ## Repartitioning Operation.
*
- * Use random round robin algorithm to evenly redistribute tuples across all target partitions, with a preference for local tasks.
- *
- * @return
+ * <p>Use random round robin algorithm to evenly redistribute tuples across all target partitions, with a preference for local tasks.
*/
public Stream localOrShuffle() {
return partition(Grouping.local_or_shuffle(new NullStruct()));
@@ -235,11 +234,9 @@
/**
- * ## Repartitioning Operation
+ * ## Repartitioning Operation.
*
- * All tuples are sent to the same partition. The same partition is chosen for all batches in the stream.
- *
- * @return
+ * <p>All tuples are sent to the same partition. The same partition is chosen for all batches in the stream.
*/
public Stream global() {
// use this instead of storm's built in one so that we can specify a singleemitbatchtopartition
@@ -248,11 +245,9 @@
}
/**
- * ## Repartitioning Operation
+ * ## Repartitioning Operation.
*
- * All tuples in the batch are sent to the same partition. Different batches in the stream may go to different partitions.
- *
- * @return
+ * <p>All tuples in the batch are sent to the same partition. Different batches in the stream may go to different partitions.
*/
public Stream batchGlobal() {
// the first field is the batch id
@@ -260,48 +255,25 @@
}
/**
- * ## Repartitioning Operation
+ * ## Repartitioning Operation.
*
- * Every tuple is replicated to all target partitions. This can useful during DRPC – for example, if you need to do a stateQuery on
+ * <p>Every tuple is replicated to all target partitions. This can useful during DRPC – for example, if you need to do a stateQuery on
* every partition of data.
- *
- * @return
*/
public Stream broadcast() {
return partition(Grouping.all(new NullStruct()));
}
/**
- * ## Repartitioning Operation
- *
- * @return
+ * ## Repartitioning Operation.
*/
public Stream identityPartition() {
return partition(new IdentityGrouping());
}
/**
- * ## Repartitioning Operation
- *
- * This method takes in a custom partitioning function that implements {@link org.apache.storm.grouping.CustomStreamGrouping}
- *
- * @param grouping
- * @return
- */
- public Stream partition(Grouping grouping) {
- if (_node instanceof PartitionNode) {
- return each(new Fields(), new TrueFilter()).partition(grouping);
- } else {
- return _topology.addSourcedNode(this, new PartitionNode(_node.streamId, _name, getOutputFields(), grouping));
- }
- }
-
- /**
* Applies an `Assembly` to this `Stream`.
*
- * @param assembly
- * @return
- *
* @see org.apache.storm.trident.operation.Assembly
*/
public Stream applyAssembly(Assembly assembly) {
@@ -311,36 +283,74 @@
@Override
public Stream each(Fields inputFields, Function function, Fields functionFields) {
projectionValidation(inputFields);
- return _topology.addSourcedNode(this,
- new ProcessorNode(_topology.getUniqueStreamId(),
- _name,
- TridentUtils.fieldsConcat(getOutputFields(), functionFields),
- functionFields,
- new EachProcessor(inputFields, function)));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ TridentUtils.fieldsConcat(getOutputFields(), functionFields),
+ functionFields,
+ new EachProcessor(inputFields, function)));
+ }
+
+ public Stream each(Function function, Fields functionFields) {
+ return each(null, function, functionFields);
+ }
+
+ public Stream each(Fields inputFields, Filter filter) {
+ return each(inputFields, new FilterExecutor(filter), new Fields());
}
//creates brand new tuples with brand new fields
@Override
public Stream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) {
projectionValidation(inputFields);
- return _topology.addSourcedNode(this,
- new ProcessorNode(_topology.getUniqueStreamId(),
- _name,
- functionFields,
- functionFields,
- new AggregateProcessor(inputFields, agg)));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ functionFields,
+ functionFields,
+ new AggregateProcessor(inputFields, agg)));
+ }
+
+ public Stream partitionAggregate(Aggregator agg, Fields functionFields) {
+ return partitionAggregate(null, agg, functionFields);
+ }
+
+ public Stream partitionAggregate(CombinerAggregator agg, Fields functionFields) {
+ return partitionAggregate(null, agg, functionFields);
+ }
+
+ public Stream partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) {
+ projectionValidation(inputFields);
+ return chainedAgg()
+ .partitionAggregate(inputFields, agg, functionFields)
+ .chainEnd();
+ }
+
+ public Stream partitionAggregate(ReducerAggregator agg, Fields functionFields) {
+ return partitionAggregate(null, agg, functionFields);
+ }
+
+ public Stream partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) {
+ projectionValidation(inputFields);
+ return chainedAgg()
+ .partitionAggregate(inputFields, agg, functionFields)
+ .chainEnd();
}
public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields) {
projectionValidation(inputFields);
- String stateId = state._node.stateInfo.id;
- Node n = new ProcessorNode(_topology.getUniqueStreamId(),
- _name,
- TridentUtils.fieldsConcat(getOutputFields(), functionFields),
- functionFields,
- new StateQueryProcessor(stateId, inputFields, function));
- _topology._colocate.get(stateId).add(n);
- return _topology.addSourcedNode(this, n);
+ String stateId = state.node.stateInfo.id;
+ Node n = new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ TridentUtils.fieldsConcat(getOutputFields(), functionFields),
+ functionFields,
+ new StateQueryProcessor(stateId, inputFields, function));
+ topology.colocate.get(stateId).add(n);
+ return topology.addSourcedNode(this, n);
+ }
+
+ public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields) {
+ return stateQuery(state, null, function, functionFields);
}
public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater, Fields functionFields) {
@@ -349,15 +359,15 @@
public TridentState partitionPersist(StateSpec stateSpec, Fields inputFields, StateUpdater updater, Fields functionFields) {
projectionValidation(inputFields);
- String id = _topology.getUniqueStateId();
- ProcessorNode n = new ProcessorNode(_topology.getUniqueStreamId(),
- _name,
- functionFields,
- functionFields,
- new PartitionPersistProcessor(id, inputFields, updater));
+ String id = topology.getUniqueStateId();
+ ProcessorNode n = new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ functionFields,
+ functionFields,
+ new PartitionPersistProcessor(id, inputFields, updater));
n.committer = true;
n.stateInfo = new NodeStateInfo(id, stateSpec);
- return _topology.addSourcedStateNode(this, n);
+ return topology.addSourcedStateNode(this, n);
}
public TridentState partitionPersist(StateFactory stateFactory, Fields inputFields, StateUpdater updater) {
@@ -368,12 +378,20 @@
return partitionPersist(stateSpec, inputFields, updater, new Fields());
}
- public Stream each(Function function, Fields functionFields) {
- return each(null, function, functionFields);
+ public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater, Fields functionFields) {
+ return partitionPersist(new StateSpec(stateFactory), updater, functionFields);
}
- public Stream each(Fields inputFields, Filter filter) {
- return each(inputFields, new FilterExecutor(filter), new Fields());
+ public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater, Fields functionFields) {
+ return partitionPersist(stateSpec, null, updater, functionFields);
+ }
+
+ public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater) {
+ return partitionPersist(stateFactory, updater, new Fields());
+ }
+
+ public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater) {
+ return partitionPersist(stateSpec, updater, new Fields());
}
/**
@@ -405,13 +423,12 @@
*/
public Stream map(MapFunction function) {
projectionValidation(getOutputFields());
- return _topology.addSourcedNode(this,
- new ProcessorNode(
- _topology.getUniqueStreamId(),
- _name,
- getOutputFields(),
- getOutputFields(),
- new MapProcessor(getOutputFields(), new MapFunctionExecutor(function))));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ getOutputFields(),
+ getOutputFields(),
+ new MapProcessor(getOutputFields(), new MapFunctionExecutor(function))));
}
/**
@@ -424,13 +441,12 @@
*/
public Stream map(MapFunction function, Fields outputFields) {
projectionValidation(getOutputFields());
- return _topology.addSourcedNode(this,
- new ProcessorNode(
- _topology.getUniqueStreamId(),
- _name,
- outputFields,
- outputFields,
- new MapProcessor(getOutputFields(), new MapFunctionExecutor(function))));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ outputFields,
+ outputFields,
+ new MapProcessor(getOutputFields(), new MapFunctionExecutor(function))));
}
/**
@@ -443,13 +459,12 @@
*/
public Stream flatMap(FlatMapFunction function) {
projectionValidation(getOutputFields());
- return _topology.addSourcedNode(this,
- new ProcessorNode(
- _topology.getUniqueStreamId(),
- _name,
- getOutputFields(),
- getOutputFields(),
- new MapProcessor(getOutputFields(), new FlatMapFunctionExecutor(function))));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ getOutputFields(),
+ getOutputFields(),
+ new MapProcessor(getOutputFields(), new FlatMapFunctionExecutor(function))));
}
/**
@@ -464,13 +479,12 @@
*/
public Stream flatMap(FlatMapFunction function, Fields outputFields) {
projectionValidation(getOutputFields());
- return _topology.addSourcedNode(this,
- new ProcessorNode(
- _topology.getUniqueStreamId(),
- _name,
- outputFields,
- outputFields,
- new MapProcessor(getOutputFields(), new FlatMapFunctionExecutor(function))));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ outputFields,
+ outputFields,
+ new MapProcessor(getOutputFields(), new FlatMapFunctionExecutor(function))));
}
/**
@@ -483,45 +497,18 @@
*/
public Stream peek(Consumer action) {
projectionValidation(getOutputFields());
- return _topology.addSourcedNode(this,
- new ProcessorNode(
- _topology.getUniqueStreamId(),
- _name,
- getOutputFields(),
- getOutputFields(),
- new MapProcessor(getOutputFields(), new ConsumerExecutor(action))));
+ return topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ getOutputFields(),
+ getOutputFields(),
+ new MapProcessor(getOutputFields(), new ConsumerExecutor(action))));
}
public ChainedAggregatorDeclarer chainedAgg() {
return new ChainedAggregatorDeclarer(this, new BatchGlobalAggScheme());
}
- public Stream partitionAggregate(Aggregator agg, Fields functionFields) {
- return partitionAggregate(null, agg, functionFields);
- }
-
- public Stream partitionAggregate(CombinerAggregator agg, Fields functionFields) {
- return partitionAggregate(null, agg, functionFields);
- }
-
- public Stream partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields) {
- projectionValidation(inputFields);
- return chainedAgg()
- .partitionAggregate(inputFields, agg, functionFields)
- .chainEnd();
- }
-
- public Stream partitionAggregate(ReducerAggregator agg, Fields functionFields) {
- return partitionAggregate(null, agg, functionFields);
- }
-
- public Stream partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields) {
- projectionValidation(inputFields);
- return chainedAgg()
- .partitionAggregate(inputFields, agg, functionFields)
- .chainEnd();
- }
-
/**
* This aggregator operation computes the minimum of tuples by the given {@code inputFieldName} and it is assumed that its value is an
* instance of {@code Comparable}. If the value of tuple with field {@code inputFieldName} is not an instance of {@code Comparable} then
@@ -658,6 +645,21 @@
}
/**
+ * Returns a stream of tuples which are aggregated results of a window that tumbles at duration of {@code windowDuration}.
+ *
+ * @param windowDuration represents tumbling window duration configuration
+ * @param windowStoreFactory intermediary tuple store for storing windowing tuples
+ * @param inputFields projected fields for aggregator
+ * @param aggregator aggregator to run on the window of tuples to compute the result and emit to the stream.
+ * @param functionFields fields of values to emit with aggregation.
+ * @return the new stream with this operation.
+ */
+ public Stream tumblingWindow(BaseWindowedBolt.Duration windowDuration, WindowsStoreFactory windowStoreFactory,
+ Fields inputFields, Aggregator aggregator, Fields functionFields) {
+ return window(TumblingDurationWindow.of(windowDuration), windowStoreFactory, inputFields, aggregator, functionFields);
+ }
+
+ /**
* Returns a stream of tuples which are aggregated results of a sliding window with every {@code windowCount} of tuples and slides the
* window after {@code slideCount}.
*
@@ -675,23 +677,8 @@
}
/**
- * Returns a stream of tuples which are aggregated results of a window that tumbles at duration of {@code windowDuration}
- *
- * @param windowDuration represents tumbling window duration configuration
- * @param windowStoreFactory intermediary tuple store for storing windowing tuples
- * @param inputFields projected fields for aggregator
- * @param aggregator aggregator to run on the window of tuples to compute the result and emit to the stream.
- * @param functionFields fields of values to emit with aggregation.
- * @return the new stream with this operation.
- */
- public Stream tumblingWindow(BaseWindowedBolt.Duration windowDuration, WindowsStoreFactory windowStoreFactory,
- Fields inputFields, Aggregator aggregator, Fields functionFields) {
- return window(TumblingDurationWindow.of(windowDuration), windowStoreFactory, inputFields, aggregator, functionFields);
- }
-
- /**
* Returns a stream of tuples which are aggregated results of a window which slides at duration of {@code slidingInterval} and completes
- * a window at {@code windowDuration}
+ * a window at {@code windowDuration}.
*
* @param windowDuration represents window duration configuration
* @param slidingInterval the time duration after which the window slides
@@ -702,9 +689,9 @@
* @return the new stream with this operation.
*/
public Stream slidingWindow(BaseWindowedBolt.Duration windowDuration, BaseWindowedBolt.Duration slidingInterval,
- WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields) {
+ WindowsStoreFactory windowStoreFactory, Fields inputFields, Aggregator aggregator, Fields functionFields) {
return window(SlidingDurationWindow.of(windowDuration, slidingInterval), windowStoreFactory, inputFields, aggregator,
- functionFields);
+ functionFields);
}
/**
@@ -747,15 +734,16 @@
// when storeTuplesInStore is false then the given windowStoreFactory is only used to store triggers and
// that store is passed to WindowStateUpdater to remove them after committing the batch.
- Stream stream = _topology.addSourcedNode(this,
- new ProcessorNode(_topology.getUniqueStreamId(),
- _name,
- fields,
- fields,
- new WindowTridentProcessor(windowConfig, _topology.getUniqueWindowId(),
- windowStoreFactory,
- inputFields, aggregator,
- storeTuplesInStore)));
+ Stream stream = topology.addSourcedNode(this,
+ new ProcessorNode(topology.getUniqueStreamId(),
+ name,
+ fields,
+ fields,
+ new WindowTridentProcessor(windowConfig,
+ topology.getUniqueWindowId(),
+ windowStoreFactory,
+ inputFields, aggregator,
+ storeTuplesInStore)));
Stream effectiveStream = stream.project(functionFields);
@@ -777,22 +765,6 @@
return new Fields(fieldsList);
}
- public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater, Fields functionFields) {
- return partitionPersist(new StateSpec(stateFactory), updater, functionFields);
- }
-
- public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater, Fields functionFields) {
- return partitionPersist(stateSpec, null, updater, functionFields);
- }
-
- public TridentState partitionPersist(StateFactory stateFactory, StateUpdater updater) {
- return partitionPersist(stateFactory, updater, new Fields());
- }
-
- public TridentState partitionPersist(StateSpec stateSpec, StateUpdater updater) {
- return partitionPersist(stateSpec, updater, new Fields());
- }
-
public TridentState persistentAggregate(StateFactory stateFactory, CombinerAggregator agg, Fields functionFields) {
return persistentAggregate(new StateSpec(stateFactory), agg, functionFields);
}
@@ -831,10 +803,6 @@
return global().partitionPersist(spec, inputFields, new ReducerAggStateUpdater(agg), functionFields);
}
- public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields) {
- return stateQuery(state, null, function, functionFields);
- }
-
@Override
public Stream toStream() {
return this;
@@ -842,7 +810,7 @@
@Override
public Fields getOutputFields() {
- return _node.allOutputFields;
+ return node.allOutputFields;
}
private void projectionValidation(Fields projFields) {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/TridentState.java b/storm-client/src/jvm/org/apache/storm/trident/TridentState.java
index ed9177b..d02ae41 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/TridentState.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/TridentState.java
@@ -18,44 +18,44 @@
public class TridentState implements ResourceDeclarer<TridentState> {
- TridentTopology _topology;
- Node _node;
+ TridentTopology topology;
+ Node node;
protected TridentState(TridentTopology topology, Node node) {
- _topology = topology;
- _node = node;
+ this.topology = topology;
+ this.node = node;
}
public Stream newValuesStream() {
- return new Stream(_topology, _node.name, _node);
+ return new Stream(topology, node.name, node);
}
public TridentState parallelismHint(int parallelism) {
- _node.parallelismHint = parallelism;
+ node.parallelismHint = parallelism;
return this;
}
@Override
public TridentState setCPULoad(Number load) {
- _node.setCPULoad(load);
+ node.setCPULoad(load);
return this;
}
@Override
public TridentState setMemoryLoad(Number onHeap) {
- _node.setMemoryLoad(onHeap);
+ node.setMemoryLoad(onHeap);
return this;
}
@Override
public TridentState setMemoryLoad(Number onHeap, Number offHeap) {
- _node.setMemoryLoad(onHeap, offHeap);
+ node.setMemoryLoad(onHeap, offHeap);
return this;
}
@Override
public TridentState addSharedMemory(SharedMemory request) {
- _node.addSharedMemory(request);
+ node.addSharedMemory(request);
return this;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/TridentTopology.java b/storm-client/src/jvm/org/apache/storm/trident/TridentTopology.java
index 336af27..cfeab4b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/TridentTopology.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/TridentTopology.java
@@ -96,11 +96,11 @@
//TODO: add a method for drpc stream, needs to know how to automatically do return results, etc
// is it too expensive to do a batch per drpc request?
- final DefaultDirectedGraph<Node, IndexedEdge> _graph;
- final Map<String, List<Node>> _colocate;
- final UniqueIdGen _gen;
- Map<String, Number> _resourceDefaults = new HashMap<>();
- Map<String, Number> _masterCoordResources = new HashMap<>();
+ final DefaultDirectedGraph<Node, IndexedEdge> graph;
+ final Map<String, List<Node>> colocate;
+ final UniqueIdGen gen;
+ Map<String, Number> resourceDefaults = new HashMap<>();
+ Map<String, Number> masterCoordResources = new HashMap<>();
public TridentTopology() {
this(new DefaultDirectedGraph<Node, IndexedEdge>(new ErrorEdgeFactory()),
@@ -109,9 +109,9 @@
}
private TridentTopology(DefaultDirectedGraph<Node, IndexedEdge> graph, Map<String, List<Node>> colocate, UniqueIdGen gen) {
- _graph = graph;
- _colocate = colocate;
- _gen = gen;
+ this.graph = graph;
+ this.colocate = colocate;
+ this.gen = gen;
}
@@ -177,7 +177,7 @@
return ret;
}
- private static void completeDRPC(DefaultDirectedGraph<Node, IndexedEdge> graph, Map<String, List<Node>> colocate, UniqueIdGen gen) {
+ private static void completeDrpc(DefaultDirectedGraph<Node, IndexedEdge> graph, Map<String, List<Node>> colocate, UniqueIdGen gen) {
List<Set<Node>> connectedComponents = new ConnectivityInspector<>(graph).connectedSets();
for (Set<Node> g : connectedComponents) {
@@ -186,7 +186,7 @@
TridentTopology helper = new TridentTopology(graph, colocate, gen);
for (Set<Node> g : connectedComponents) {
- SpoutNode drpcNode = getDRPCSpoutNode(g);
+ SpoutNode drpcNode = getDrpcSpoutNode(g);
if (drpcNode != null) {
Stream lastStream = new Stream(helper, null, getLastAddedNode(g));
Stream s = new Stream(helper, null, drpcNode);
@@ -211,7 +211,7 @@
}
//returns null if it's not a drpc group
- private static SpoutNode getDRPCSpoutNode(Collection<Node> g) {
+ private static SpoutNode getDrpcSpoutNode(Collection<Node> g) {
for (Node n : g) {
if (n instanceof SpoutNode) {
SpoutNode.SpoutType type = ((SpoutNode) n).type;
@@ -224,7 +224,7 @@
}
private static void checkValidJoins(Collection<Node> g) {
- boolean hasDRPCSpout = false;
+ boolean hasDrpcSpout = false;
boolean hasBatchSpout = false;
for (Node n : g) {
if (n instanceof SpoutNode) {
@@ -232,11 +232,11 @@
if (type == SpoutNode.SpoutType.BATCH) {
hasBatchSpout = true;
} else if (type == SpoutNode.SpoutType.DRPC) {
- hasDRPCSpout = true;
+ hasDrpcSpout = true;
}
}
}
- if (hasBatchSpout && hasDRPCSpout) {
+ if (hasBatchSpout && hasDrpcSpout) {
throw new RuntimeException("Cannot join DRPC stream with streams originating from other spouts");
}
}
@@ -250,9 +250,8 @@
for (PartitionNode n : subscriptions) {
PartitionNode curr = ret.get(n.streamId);
if (curr != null && !curr.thriftGrouping.equals(n.thriftGrouping)) {
- throw new RuntimeException(
- "Multiple subscriptions to the same stream with different groupings. Should be impossible since that is explicitly " +
- "guarded against.");
+ throw new RuntimeException("Multiple subscriptions to the same stream with different groupings. Should "
+ + "be impossible since that is explicitly guarded against.");
}
ret.put(n.streamId, n);
}
@@ -560,10 +559,12 @@
}
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Stream newDRPCStream(String function) {
return newDRPCStream(new DRPCSpout(function));
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public Stream newDRPCStream(String function, ILocalDRPC server) {
DRPCSpout spout;
if (server == null) {
@@ -574,6 +575,7 @@
return newDRPCStream(spout);
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
private Stream newDRPCStream(DRPCSpout spout) {
// TODO: consider adding a shuffle grouping after the spout to avoid so much routing of the args/return-info all over the place
// (at least until its possible to just pack bolt logic into the spout itself)
@@ -624,8 +626,8 @@
public Stream multiReduce(List<Fields> inputFields, List<Stream> streams, MultiReducer function, Fields outputFields) {
List<String> names = new ArrayList<>();
for (Stream s : streams) {
- if (s._name != null) {
- names.add(s._name);
+ if (s.name != null) {
+ names.add(s.name);
}
}
Node n = new ProcessorNode(getUniqueStreamId(), Utils.join(names, "-"), outputFields, outputFields,
@@ -735,20 +737,20 @@
}
public TridentTopology setResourceDefaults(DefaultResourceDeclarer defaults) {
- _resourceDefaults = defaults.getResources();
+ resourceDefaults = defaults.getResources();
return this;
}
public TridentTopology setMasterCoordResources(DefaultResourceDeclarer resources) {
- _masterCoordResources = resources.getResources();
+ masterCoordResources = resources.getResources();
return this;
}
public StormTopology build() {
- DefaultDirectedGraph<Node, IndexedEdge> graph = (DefaultDirectedGraph) _graph.clone();
+ DefaultDirectedGraph<Node, IndexedEdge> graph = (DefaultDirectedGraph) this.graph.clone();
- completeDRPC(graph, _colocate, _gen);
+ completeDrpc(graph, colocate, gen);
List<SpoutNode> spoutNodes = new ArrayList<>();
@@ -764,7 +766,7 @@
Set<Group> initialGroups = new LinkedHashSet<>();
- for (List<Node> colocate : _colocate.values()) {
+ for (List<Node> colocate : colocate.values()) {
Group g = new Group(graph, colocate);
boltNodes.removeAll(colocate);
initialGroups.add(g);
@@ -790,10 +792,10 @@
}
if (g1 == null || !g1.equals(g2)) {
graph.removeEdge(e);
- PartitionNode pNode = makeIdentityPartition(e.source);
- graph.addVertex(pNode);
- graph.addEdge(e.source, pNode, new IndexedEdge(e.source, pNode, 0));
- graph.addEdge(pNode, e.target, new IndexedEdge(pNode, e.target, e.index));
+ PartitionNode partitionNode = makeIdentityPartition(e.source);
+ graph.addVertex(partitionNode);
+ graph.addEdge(e.source, partitionNode, new IndexedEdge(e.source, partitionNode, 0));
+ graph.addEdge(partitionNode, e.target, new IndexedEdge(partitionNode, e.target, e.index));
}
}
}
@@ -810,14 +812,14 @@
for (PartitionNode n : extraPartitionInputs(g)) {
Node idNode = makeIdentityNode(n.allOutputFields);
Node newPartitionNode = new PartitionNode(idNode.streamId, n.name, idNode.allOutputFields, n.thriftGrouping);
- Node parentNode = TridentUtils.getParent(graph, n);
- Set<IndexedEdge> outgoing = graph.outgoingEdgesOf(n);
graph.removeVertex(n);
graph.addVertex(idNode);
graph.addVertex(newPartitionNode);
+ Node parentNode = TridentUtils.getParent(graph, n);
addEdge(graph, parentNode, idNode, 0);
addEdge(graph, idNode, newPartitionNode, 0);
+ Set<IndexedEdge> outgoing = graph.outgoingEdgesOf(n);
for (IndexedEdge e : outgoing) {
addEdge(graph, newPartitionNode, e.target, e.index);
}
@@ -868,7 +870,7 @@
for (SpoutNode sn : spoutNodes) {
Integer parallelism = parallelisms.get(grouper.nodeGroup(sn));
- Map<String, Number> spoutRes = new HashMap<>(_resourceDefaults);
+ Map<String, Number> spoutRes = new HashMap<>(resourceDefaults);
spoutRes.putAll(sn.getResources());
Number onHeap = spoutRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
@@ -911,7 +913,7 @@
if (!isSpoutGroup(g)) {
Integer p = parallelisms.get(g);
Map<String, String> streamToGroup = getOutputStreamBatchGroups(g, batchGroupMap);
- Map<String, Number> groupRes = g.getResources(_resourceDefaults);
+ Map<String, Number> groupRes = g.getResources(resourceDefaults);
Number onHeap = groupRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
Number offHeap = groupRes.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB);
@@ -939,14 +941,15 @@
Collection<PartitionNode> inputs = uniquedSubscriptions(externalGroupInputs(g));
for (PartitionNode n : inputs) {
Node parent = TridentUtils.getParent(graph, n);
- String componentId = parent instanceof SpoutNode ?
- spoutIds.get(parent) : boltIds.get(grouper.nodeGroup(parent));
+ String componentId = parent instanceof SpoutNode
+ ? spoutIds.get(parent)
+ : boltIds.get(grouper.nodeGroup(parent));
d.grouping(new GlobalStreamId(componentId, n.streamId), n.thriftGrouping);
}
}
}
- HashMap<String, Number> combinedMasterCoordResources = new HashMap<>(_resourceDefaults);
- combinedMasterCoordResources.putAll(_masterCoordResources);
+ HashMap<String, Number> combinedMasterCoordResources = new HashMap<>(resourceDefaults);
+ combinedMasterCoordResources.putAll(masterCoordResources);
return builder.buildTopology(combinedMasterCoordResources);
}
@@ -956,25 +959,25 @@
}
protected String getUniqueStreamId() {
- return _gen.getUniqueStreamId();
+ return gen.getUniqueStreamId();
}
protected String getUniqueStateId() {
- return _gen.getUniqueStateId();
+ return gen.getUniqueStateId();
}
protected String getUniqueWindowId() {
- return _gen.getUniqueWindowId();
+ return gen.getUniqueWindowId();
}
protected void registerNode(Node n) {
- _graph.addVertex(n);
+ graph.addVertex(n);
if (n.stateInfo != null) {
String id = n.stateInfo.id;
- if (!_colocate.containsKey(id)) {
- _colocate.put(id, new ArrayList());
+ if (!colocate.containsKey(id)) {
+ colocate.put(id, new ArrayList());
}
- _colocate.get(id).add(n);
+ colocate.get(id).add(n);
}
}
@@ -987,7 +990,7 @@
registerNode(newNode);
int streamIndex = 0;
for (Stream s : sources) {
- _graph.addEdge(s._node, newNode, new IndexedEdge(s._node, newNode, streamIndex));
+ graph.addEdge(s.node, newNode, new IndexedEdge(s.node, newNode, streamIndex));
streamIndex++;
}
}
@@ -997,15 +1000,15 @@
return new Stream(this, newNode.name, newNode);
}
+ protected Stream addSourcedNode(Stream source, Node newNode) {
+ return addSourcedNode(Arrays.asList(source), newNode);
+ }
+
protected TridentState addSourcedStateNode(List<Stream> sources, Node newNode) {
registerSourcedNode(sources, newNode);
return new TridentState(this, newNode);
}
- protected Stream addSourcedNode(Stream source, Node newNode) {
- return addSourcedNode(Arrays.asList(source), newNode);
- }
-
protected TridentState addSourcedStateNode(Stream source, Node newNode) {
return addSourcedStateNode(Arrays.asList(source), newNode);
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/drpc/ReturnResultsReducer.java b/storm-client/src/jvm/org/apache/storm/trident/drpc/ReturnResultsReducer.java
index 2af7498..77c0e12 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/drpc/ReturnResultsReducer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/drpc/ReturnResultsReducer.java
@@ -36,7 +36,7 @@
public class ReturnResultsReducer implements MultiReducer<ReturnResultsState> {
boolean local;
Map<String, Object> conf;
- Map<List, DRPCInvocationsClient> _clients = new HashMap<>();
+ Map<List, DRPCInvocationsClient> clients = new HashMap<>();
@Override
public void prepare(Map<String, Object> conf, TridentMultiReducerContext context) {
@@ -77,19 +77,21 @@
if (local) {
client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host);
} else {
- List server = new ArrayList() {{
- add(host);
- add(port);
- }};
+ List server = new ArrayList() {
+ {
+ add(host);
+ add(port);
+ }
+ };
- if (!_clients.containsKey(server)) {
+ if (!clients.containsKey(server)) {
try {
- _clients.put(server, new DRPCInvocationsClient(conf, host, port));
+ clients.put(server, new DRPCInvocationsClient(conf, host, port));
} catch (TTransportException ex) {
throw new RuntimeException(ex);
}
}
- client = _clients.get(server);
+ client = clients.get(server);
}
try {
@@ -104,7 +106,7 @@
@Override
public void cleanup() {
- for (DRPCInvocationsClient c : _clients.values()) {
+ for (DRPCInvocationsClient c : clients.values()) {
c.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/fluent/ChainedAggregatorDeclarer.java b/storm-client/src/jvm/org/apache/storm/trident/fluent/ChainedAggregatorDeclarer.java
index 51d8ab7..81ec21c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/fluent/ChainedAggregatorDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/fluent/ChainedAggregatorDeclarer.java
@@ -31,25 +31,25 @@
public class ChainedAggregatorDeclarer implements ChainedFullAggregatorDeclarer, ChainedPartitionAggregatorDeclarer {
- List<AggSpec> _aggs = new ArrayList<>();
- IAggregatableStream _stream;
- AggType _type = null;
- GlobalAggregationScheme _globalScheme;
+ List<AggSpec> aggs = new ArrayList<>();
+ IAggregatableStream stream;
+ AggType type = null;
+ GlobalAggregationScheme globalScheme;
public ChainedAggregatorDeclarer(IAggregatableStream stream, GlobalAggregationScheme globalScheme) {
- _stream = stream;
- _globalScheme = globalScheme;
+ this.stream = stream;
+ this.globalScheme = globalScheme;
}
@Override
public Stream chainEnd() {
- Fields[] inputFields = new Fields[_aggs.size()];
- Aggregator[] aggs = new Aggregator[_aggs.size()];
- int[] outSizes = new int[_aggs.size()];
+ Fields[] inputFields = new Fields[aggs.size()];
+ Aggregator[] aggs = new Aggregator[this.aggs.size()];
+ int[] outSizes = new int[this.aggs.size()];
List<String> allOutFields = new ArrayList<>();
Set<String> allInFields = new HashSet<>();
- for (int i = 0; i < _aggs.size(); i++) {
- AggSpec spec = _aggs.get(i);
+ for (int i = 0; i < this.aggs.size(); i++) {
+ AggSpec spec = this.aggs.get(i);
Fields infields = spec.inFields;
if (infields == null) {
infields = new Fields();
@@ -73,21 +73,21 @@
Fields outFields = new Fields(allOutFields);
Aggregator combined = new ChainedAggregatorImpl(aggs, inputFields, new ComboList.Factory(outSizes));
- if (_type != AggType.FULL) {
- _stream = _stream.partitionAggregate(inFields, combined, outFields);
+ if (type != AggType.FULL) {
+ stream = stream.partitionAggregate(inFields, combined, outFields);
}
- if (_type != AggType.PARTITION) {
- _stream = _globalScheme.aggPartition(_stream);
- BatchToPartition singleEmit = _globalScheme.singleEmitPartitioner();
+ if (type != AggType.PARTITION) {
+ stream = globalScheme.aggPartition(stream);
+ BatchToPartition singleEmit = globalScheme.singleEmitPartitioner();
Aggregator toAgg = combined;
if (singleEmit != null) {
toAgg = new SingleEmitAggregator(combined, singleEmit);
}
// this assumes that inFields and outFields are the same for combineragg
// assumption also made above
- _stream = _stream.partitionAggregate(inFields, toAgg, outFields);
+ stream = stream.partitionAggregate(inFields, toAgg, outFields);
}
- return _stream.toStream();
+ return stream.toStream();
}
@Override
@@ -97,8 +97,8 @@
@Override
public ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) {
- _type = AggType.PARTITION;
- _aggs.add(new AggSpec(inputFields, agg, functionFields));
+ type = AggType.PARTITION;
+ aggs.add(new AggSpec(inputFields, agg, functionFields));
return this;
}
@@ -135,13 +135,13 @@
private ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields, boolean isCombiner) {
if (isCombiner) {
- if (_type == null) {
- _type = AggType.FULL_COMBINE;
+ if (type == null) {
+ type = AggType.FULL_COMBINE;
}
} else {
- _type = AggType.FULL;
+ type = AggType.FULL;
}
- _aggs.add(new AggSpec(inputFields, agg, functionFields));
+ aggs.add(new AggSpec(inputFields, agg, functionFields));
return this;
}
@@ -167,7 +167,7 @@
}
private void initCombiner(Fields inputFields, CombinerAggregator agg, Fields functionFields) {
- _stream = _stream.each(inputFields, new CombinerAggregatorInitImpl(agg), functionFields);
+ stream = stream.each(inputFields, new CombinerAggregatorInitImpl(agg), functionFields);
}
private static enum AggType {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/fluent/GroupedStream.java b/storm-client/src/jvm/org/apache/storm/trident/fluent/GroupedStream.java
index 1a84dcd..2c8dcf3 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/fluent/GroupedStream.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/fluent/GroupedStream.java
@@ -30,16 +30,16 @@
public class GroupedStream implements IAggregatableStream, GlobalAggregationScheme<GroupedStream> {
- Fields _groupFields;
- Stream _stream;
+ Fields groupFields;
+ Stream stream;
public GroupedStream(Stream stream, Fields groupFields) {
- _groupFields = groupFields;
- _stream = stream;
+ this.groupFields = groupFields;
+ this.stream = stream;
}
public GroupedStream name(String name) {
- return new GroupedStream(_stream.name(name), _groupFields);
+ return new GroupedStream(stream.name(name), groupFields);
}
public ChainedAggregatorDeclarer chainedAgg() {
@@ -91,9 +91,9 @@
public TridentState persistentAggregate(StateSpec spec, Fields inputFields, CombinerAggregator agg, Fields functionFields) {
return aggregate(inputFields, agg, functionFields)
.partitionPersist(spec,
- TridentUtils.fieldsUnion(_groupFields, functionFields),
- new MapCombinerAggStateUpdater(agg, _groupFields, functionFields),
- TridentUtils.fieldsConcat(_groupFields, functionFields));
+ TridentUtils.fieldsUnion(groupFields, functionFields),
+ new MapCombinerAggStateUpdater(agg, groupFields, functionFields),
+ TridentUtils.fieldsConcat(groupFields, functionFields));
}
public TridentState persistentAggregate(StateFactory stateFactory, Fields inputFields, ReducerAggregator agg, Fields functionFields) {
@@ -101,19 +101,11 @@
}
public TridentState persistentAggregate(StateSpec spec, Fields inputFields, ReducerAggregator agg, Fields functionFields) {
- return _stream.partitionBy(_groupFields)
+ return stream.partitionBy(groupFields)
.partitionPersist(spec,
- TridentUtils.fieldsUnion(_groupFields, inputFields),
- new MapReducerAggStateUpdater(agg, _groupFields, inputFields),
- TridentUtils.fieldsConcat(_groupFields, functionFields));
- }
-
- public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields) {
- return _stream.partitionBy(_groupFields)
- .stateQuery(state,
- inputFields,
- function,
- functionFields);
+ TridentUtils.fieldsUnion(groupFields, inputFields),
+ new MapReducerAggStateUpdater(agg, groupFields, inputFields),
+ TridentUtils.fieldsConcat(groupFields, functionFields));
}
public TridentState persistentAggregate(StateFactory stateFactory, ReducerAggregator agg, Fields functionFields) {
@@ -124,42 +116,50 @@
return persistentAggregate(spec, null, agg, functionFields);
}
+ public Stream stateQuery(TridentState state, Fields inputFields, QueryFunction function, Fields functionFields) {
+ return stream.partitionBy(groupFields)
+ .stateQuery(state,
+ inputFields,
+ function,
+ functionFields);
+ }
+
public Stream stateQuery(TridentState state, QueryFunction function, Fields functionFields) {
return stateQuery(state, null, function, functionFields);
}
@Override
public IAggregatableStream each(Fields inputFields, Function function, Fields functionFields) {
- Stream s = _stream.each(inputFields, function, functionFields);
- return new GroupedStream(s, _groupFields);
+ Stream s = stream.each(inputFields, function, functionFields);
+ return new GroupedStream(s, groupFields);
}
@Override
public IAggregatableStream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields) {
- Aggregator groupedAgg = new GroupedAggregator(agg, _groupFields, inputFields, functionFields.size());
- Fields allInFields = TridentUtils.fieldsUnion(_groupFields, inputFields);
- Fields allOutFields = TridentUtils.fieldsConcat(_groupFields, functionFields);
- Stream s = _stream.partitionAggregate(allInFields, groupedAgg, allOutFields);
- return new GroupedStream(s, _groupFields);
+ Aggregator groupedAgg = new GroupedAggregator(agg, groupFields, inputFields, functionFields.size());
+ Fields allInFields = TridentUtils.fieldsUnion(groupFields, inputFields);
+ Fields allOutFields = TridentUtils.fieldsConcat(groupFields, functionFields);
+ Stream s = stream.partitionAggregate(allInFields, groupedAgg, allOutFields);
+ return new GroupedStream(s, groupFields);
}
@Override
public IAggregatableStream aggPartition(GroupedStream s) {
- return new GroupedStream(s._stream.partitionBy(_groupFields), _groupFields);
+ return new GroupedStream(s.stream.partitionBy(groupFields), groupFields);
}
@Override
public Stream toStream() {
- return _stream;
+ return stream;
}
@Override
public Fields getOutputFields() {
- return _stream.getOutputFields();
+ return stream.getOutputFields();
}
public Fields getGroupFields() {
- return _groupFields;
+ return groupFields;
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/fluent/UniqueIdGen.java b/storm-client/src/jvm/org/apache/storm/trident/fluent/UniqueIdGen.java
index 4ed3f23..f04e76a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/fluent/UniqueIdGen.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/fluent/UniqueIdGen.java
@@ -13,18 +13,18 @@
package org.apache.storm.trident.fluent;
public class UniqueIdGen {
- private int _streamCounter = 0;
- private int _stateCounter = 0;
+ private int streamCounter = 0;
+ private int stateCounter = 0;
private int windowCounter = 0;
public String getUniqueStreamId() {
- _streamCounter++;
- return "s" + _streamCounter;
+ streamCounter++;
+ return "s" + streamCounter;
}
public String getUniqueStateId() {
- _stateCounter++;
- return "state" + _stateCounter;
+ stateCounter++;
+ return "state" + stateCounter;
}
public String getUniqueWindowId() {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/graph/Group.java b/storm-client/src/jvm/org/apache/storm/trident/graph/Group.java
index 681b429..456f1a0 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/graph/Group.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/graph/Group.java
@@ -62,6 +62,7 @@
}
/**
+ * Get shared memory.
* @return the shared memory requests for the entire group
*/
public Set<SharedMemory> getSharedMemory() {
@@ -90,42 +91,42 @@
resources = new HashMap<>(defaults);
resources.putAll(n.getResources());
} else {
- Map<String, Number> node_res = new HashMap<>(defaults);
- node_res.putAll(n.getResources());
+ Map<String, Number> nodeRes = new HashMap<>(defaults);
+ nodeRes.putAll(n.getResources());
- if (!node_res.keySet().equals(resources.keySet())) {
+ if (!nodeRes.keySet().equals(resources.keySet())) {
StringBuilder ops = new StringBuilder();
for (Node nod : nodes) {
- Set<String> resource_keys = new HashSet<>(defaults.keySet());
- resource_keys.addAll(nod.getResources().keySet());
- ops.append("\t[ " + nod.shortString() + ", Resources Set: " + resource_keys + " ]\n");
+ Set<String> resourceKeys = new HashSet<>(defaults.keySet());
+ resourceKeys.addAll(nod.getResources().keySet());
+ ops.append("\t[ " + nod.shortString() + ", Resources Set: " + resourceKeys + " ]\n");
}
- if (node_res.keySet().containsAll(resources.keySet())) {
- Set<String> diffset = new HashSet<>(node_res.keySet());
+ if (nodeRes.keySet().containsAll(resources.keySet())) {
+ Set<String> diffset = new HashSet<>(nodeRes.keySet());
diffset.removeAll(resources.keySet());
throw new RuntimeException(
- "Found an operation with resources set which are not set in other operations in the group:\n" +
- "\t[ " + n.shortString() + " ]: " + diffset + "\n" +
- "Either set these resources in all other operations in the group, add a default setting, or remove the " +
- "setting from this operation.\n" +
- "The group at fault:\n" +
- ops);
- } else if (resources.keySet().containsAll(node_res.keySet())) {
+ "Found an operation with resources set which are not set in other operations in the group:\n"
+ + "\t[ " + n.shortString() + " ]: " + diffset + "\n"
+ + "Either set these resources in all other operations in the group, add a default "
+ + "setting, or remove the setting from this operation.\n"
+ + "The group at fault:\n"
+ + ops);
+ } else if (resources.keySet().containsAll(nodeRes.keySet())) {
Set<String> diffset = new HashSet<>(resources.keySet());
- diffset.removeAll(node_res.keySet());
+ diffset.removeAll(nodeRes.keySet());
throw new RuntimeException(
- "Found an operation with resources unset which are set in other operations in the group:\n" +
- "\t[ " + n.shortString() + " ]: " + diffset + "\n" +
- "Either set these resources in all other operations in the group, add a default setting, or remove the " +
- "setting from all other operations.\n" +
- "The group at fault:\n" +
- ops);
+ "Found an operation with resources unset which are set in other operations in the group:\n"
+ + "\t[ " + n.shortString() + " ]: " + diffset + "\n"
+ + "Either set these resources in all other operations in the group, add a default "
+ + "setting, or remove the setting from all other operations.\n"
+ + "The group at fault:\n"
+ + ops);
}
}
- for (Map.Entry<String, Number> kv : node_res.entrySet()) {
+ for (Map.Entry<String, Number> kv : nodeRes.entrySet()) {
String key = kv.getKey();
Number val = kv.getValue();
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/Assembly.java b/storm-client/src/jvm/org/apache/storm/trident/operation/Assembly.java
index f291d9b..d4d7172 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/Assembly.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/Assembly.java
@@ -18,19 +18,16 @@
/**
* The `Assembly` interface provides a means to encapsulate logic applied to a {@link org.apache.storm.trident.Stream}.
*
- * Usage:
+ * <p>Usage:
*
- * ```java Stream mystream = ...; Stream assemblyStream = mystream.applyAssembly(myAssembly); ```
+ * <p>```java Stream mystream = ...; Stream assemblyStream = mystream.applyAssembly(myAssembly); ```
*
* @see org.apache.storm.trident.Stream
* @see org.apache.storm.trident.operation.builtin.FirstN
*/
public interface Assembly {
/**
- * Applies the `Assembly` to a given {@link org.apache.storm.trident.Stream}
- *
- * @param input
- * @return
+ * Applies the `Assembly` to a given {@link org.apache.storm.trident.Stream}.
*/
Stream apply(Stream input);
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/BaseOperation.java b/storm-client/src/jvm/org/apache/storm/trident/operation/BaseOperation.java
index d157aa4..413ede4 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/BaseOperation.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/BaseOperation.java
@@ -17,7 +17,7 @@
/**
* Convenience implementation of the {@link org.apache.storm.trident.operation.Operation} interface.
*
- * Provides no-op implementations of the `prepare()` and `cleanup()` methods.
+ * <p>Provides no-op implementations of the `prepare()` and `cleanup()` methods.
*/
public class BaseOperation implements Operation {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java b/storm-client/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java
index f5c9805..f21d59c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/DefaultResourceDeclarer.java
@@ -21,8 +21,9 @@
import org.apache.storm.topology.ResourceDeclarer;
/**
- * @param T Must always be the type of the extending class. i.e. public class SubResourceDeclarer extends
- * DefaultResourceDeclarer<SubResourceDeclarer> {...}
+ * Default implementation of resources declarer.
+ * @param <T> Must always be the type of the extending class. i.e. <code>public class SubResourceDeclarer extends
+ * DefaultResourceDeclarer<SubResourceDeclarer> {...}</code>
*/
public class DefaultResourceDeclarer<T extends DefaultResourceDeclarer> implements ResourceDeclarer<T>, ITridentResource {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/Filter.java b/storm-client/src/jvm/org/apache/storm/trident/operation/Filter.java
index 746cef9..a6f0940 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/Filter.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/Filter.java
@@ -18,10 +18,10 @@
/**
* Filters take in a tuple as input and decide whether or not to keep that tuple or not.
*
- * If the `isKeep()` method of a Filter returns `false` for a tuple, that tuple will be filtered out of the Stream
+ * <p>If the `isKeep()` method of a Filter returns `false` for a tuple, that tuple will be filtered out of the Stream
*
*
- * ### Configuration If your `Filter` implementation has configuration requirements, you will typically want to extend {@link
+ * <p>### Configuration If your `Filter` implementation has configuration requirements, you will typically want to extend {@link
* org.apache.storm.trident.operation.BaseFilter} and override the {@link org.apache.storm.trident.operation.Operation#prepare(Map,
* TridentOperationContext)} method to perform your custom initialization.
*
@@ -30,7 +30,7 @@
public interface Filter extends EachOperation {
/**
- * Determines if a tuple should be filtered out of a stream
+ * Determines if a tuple should be filtered out of a stream.
*
* @param tuple the tuple being evaluated
* @return `false` to drop the tuple, `true` to keep the tuple
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/FlatMapFunction.java b/storm-client/src/jvm/org/apache/storm/trident/operation/FlatMapFunction.java
index 0451f29..341f98b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/FlatMapFunction.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/FlatMapFunction.java
@@ -17,7 +17,7 @@
import org.apache.storm.tuple.Values;
/**
- * A one to many transformation function
+ * A one to many transformation function.
*/
public interface FlatMapFunction extends Serializable {
/**
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/Function.java b/storm-client/src/jvm/org/apache/storm/trident/operation/Function.java
index b501815..5f4dd63 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/Function.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/Function.java
@@ -20,36 +20,36 @@
* original input tuple in the stream. If a function emits no tuples, the original input tuple is filtered out. Otherwise, the input tuple
* is duplicated for each output tuple.
*
- * For example, if you have the following function:
+ * <p>For example, if you have the following function:
*
- * ```java public class MyFunction extends BaseFunction { public void execute(TridentTuple tuple, TridentCollector collector) { for(int i=0;
- * i < tuple.getInteger(0); i++) { collector.emit(new Values(i)); } } }
- *
+ * <p>```java public class MyFunction extends BaseFunction { public void execute(TridentTuple tuple, TridentCollector collector)
+ * { for(int i=0; i < tuple.getInteger(0); i++) { collector.emit(new Values(i)); } } }
* ```
*
- * Now suppose you have a stream in the variable `mystream` with the fields `["a", "b", "c"]` with the following tuples:
+ * <p>Now suppose you have a stream in the variable `mystream` with the fields `["a", "b", "c"]` with the following tuples:
*
- * ``` [1, 2, 3] [4, 1, 6] [3, 0, 8] ``` If you had the following code in your topology definition:
+ * <p>``` [1, 2, 3] [4, 1, 6] [3, 0, 8] ``` If you had the following code in your topology definition:
*
- * ```java mystream.each(new Fields("b"), new MyFunction(), new Fields("d"))) ```
+ * <p>```java mystream.each(new Fields("b"), new MyFunction(), new Fields("d"))) ```
*
- * The resulting tuples would have the fields `["a", "b", "c", "d"]` and look like this:
+ * <p>The resulting tuples would have the fields `["a", "b", "c", "d"]` and look like this:
*
- * ``` [1, 2, 3, 0] [1, 2, 3, 1] [4, 1, 6, 0] ```
+ * <p>``` [1, 2, 3, 0] [1, 2, 3, 1] [4, 1, 6, 0] ```
*
- * In this case, the parameter `new Fields("b")` tells Trident that you would like to select the field "b" as input to the function, and
+ * <p>In this case, the parameter `new Fields("b")` tells Trident that you would like to select the field "b" as input to the function, and
* that will be the only field in the Tuple passed to the `execute()` method. The value of "b" in the first tuple (2) causes the for loop to
* execute twice, so 2 tuples are emitted. similarly the second tuple causes one tuple to be emitted. For the third tuple, the value of 0
* causes the `for` loop to be skipped, so nothing is emitted and the incoming tuple is filtered out of the stream.
*
- * ### Configuration If your `Function` implementation has configuration requirements, you will typically want to extend {@link
+ * <p>### Configuration If your `Function` implementation has configuration requirements, you will typically want to extend {@link
* org.apache.storm.trident.operation.BaseFunction} and override the {@link org.apache.storm.trident.operation.Operation#prepare(Map,
* TridentOperationContext)} method to perform your custom initialization.
*
- * ### Performance Considerations Because Trident Functions perform logic on individual tuples -- as opposed to batches -- it is advisable
- * to avoid expensive operations such as database operations in a Function, if possible. For data store interactions it is better to use a
- * {@link org.apache.storm.trident.state.State} or {@link org.apache.storm.trident.state.QueryFunction} implementation since Trident states
- * operate on batch partitions and can perform bulk updates to a database.
+ * <p>### Performance Considerations Because Trident Functions perform logic on individual tuples -- as opposed to
+ * batches -- it is advisable to avoid expensive operations such as database operations in a Function, if possible. For
+ * data store interactions it is better to use a {@link org.apache.storm.trident.state.State} or
+ * {@link org.apache.storm.trident.state.QueryFunction} implementation since Trident states operate on batch partitions
+ * and can perform bulk updates to a database.
*/
public interface Function extends EachOperation {
/**
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/ITridentResource.java b/storm-client/src/jvm/org/apache/storm/trident/operation/ITridentResource.java
index 78abb9e..8fd17f4 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/ITridentResource.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/ITridentResource.java
@@ -23,11 +23,13 @@
*/
public interface ITridentResource {
/**
+ * Get resource.
* @return a name of resource name -> amount of that resource. *Return should never be null!*
*/
Map<String, Number> getResources();
/**
+ * Get shared memory.
* @return the shared memory region requests
*/
Set<SharedMemory> getSharedMemory();
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/MapFunction.java b/storm-client/src/jvm/org/apache/storm/trident/operation/MapFunction.java
index f27dfaa..300d2b9 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/MapFunction.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/MapFunction.java
@@ -17,7 +17,7 @@
import org.apache.storm.tuple.Values;
/**
- * A one-one transformation function
+ * A one-one transformation function.
*/
public interface MapFunction extends Serializable {
/**
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/Operation.java b/storm-client/src/jvm/org/apache/storm/trident/operation/Operation.java
index e119169..d3e1e7a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/Operation.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/Operation.java
@@ -18,7 +18,7 @@
/**
* Parent interface for Trident `Filter`s and `Function`s.
*
- * `Operation` defines two lifecycle methods for Trident components. The `prepare()` method is called once when the `Operation` is first
+ * <p>`Operation` defines two lifecycle methods for Trident components. The `prepare()` method is called once when the `Operation` is first
* initialized. The `cleanup()` method is called in local mode when the local cluster is being shut down. In distributed mode, the
* `cleanup()` method is not guaranteed to be called in every situation, but Storm will make a best effort call `cleanup()` whenever
* possible.
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareFlatMapFunction.java b/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareFlatMapFunction.java
index 9a4c44c..eebcf5d 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareFlatMapFunction.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareFlatMapFunction.java
@@ -13,7 +13,7 @@
package org.apache.storm.trident.operation;
/**
- * A one to many transformation function which is aware of Operation (lifecycle of the Trident component)
+ * A one to many transformation function which is aware of Operation (lifecycle of the Trident component).
*/
public interface OperationAwareFlatMapFunction extends FlatMapFunction, Operation {
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareMapFunction.java b/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareMapFunction.java
index fc3551d..7fa4a71 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareMapFunction.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/OperationAwareMapFunction.java
@@ -13,7 +13,7 @@
package org.apache.storm.trident.operation;
/**
- * A one-one transformation function which is aware of Operation (lifecycle of the Trident component)
+ * A one-one transformation function which is aware of Operation (lifecycle of the Trident component).
*/
public interface OperationAwareMapFunction extends MapFunction, Operation {
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentCollector.java b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentCollector.java
index 1db91ec..d61880b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentCollector.java
@@ -18,33 +18,33 @@
/**
* Interface for publishing tuples to a stream and reporting exceptions (to be displayed in Storm UI).
*
- * Trident components that have the ability to emit tuples to a stream are passed an instance of this interface.
+ * <p>Trident components that have the ability to emit tuples to a stream are passed an instance of this interface.
*
- * For example, to emit a new tuple to a stream, you would do something like the following:
+ * <p>For example, to emit a new tuple to a stream, you would do something like the following:
*
- * ```java collector.emit(new Values("a", "b", "c")); ```
+ * <p>```java collector.emit(new Values("a", "b", "c")); ```
*
* @see org.apache.storm.trident.Stream
* @see org.apache.storm.tuple.Values
*/
public interface TridentCollector {
/**
- * Emits a tuple to a Stream
+ * Emits a tuple to a Stream.
*
* @param values a list of values of which the tuple will be composed
*/
void emit(List<Object> values);
/**
- * Flush any buffered tuples (when batching is enabled)
+ * Flush any buffered tuples (when batching is enabled).
*/
void flush();
/**
* Reports an error. The corresponding stack trace will be visible in the Storm UI.
*
- * Note that calling this method does not alter the processing of a batch. To explicitly fail a batch and trigger a replay, components
- * should throw {@link org.apache.storm.topology.FailedException}.
+ * <p>Note that calling this method does not alter the processing of a batch. To explicitly fail a batch and trigger
+ * a replay, components should throw {@link org.apache.storm.topology.FailedException}.
*
* @param t The instance of the error (Throwable) being reported.
*/
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentMultiReducerContext.java b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentMultiReducerContext.java
index 476d5fa..21409d2 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentMultiReducerContext.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentMultiReducerContext.java
@@ -19,13 +19,13 @@
public class TridentMultiReducerContext {
- List<TridentTuple.Factory> _factories;
+ List<TridentTuple.Factory> factories;
public TridentMultiReducerContext(List<TridentTuple.Factory> factories) {
- _factories = factories;
+ this.factories = factories;
}
public ProjectionFactory makeProjectionFactory(int streamIndex, Fields fields) {
- return new ProjectionFactory(_factories.get(streamIndex), fields);
+ return new ProjectionFactory(factories.get(streamIndex), fields);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentOperationContext.java b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentOperationContext.java
index 473eda6..68111a6 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/TridentOperationContext.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/TridentOperationContext.java
@@ -29,67 +29,67 @@
import org.apache.storm.tuple.Fields;
public class TridentOperationContext implements IMetricsContext {
- TridentTuple.Factory _factory;
- TopologyContext _topoContext;
+ TridentTuple.Factory factory;
+ TopologyContext topoContext;
public TridentOperationContext(TopologyContext topoContext, TridentTuple.Factory factory) {
- _factory = factory;
- _topoContext = topoContext;
+ this.factory = factory;
+ this.topoContext = topoContext;
}
public TridentOperationContext(TridentOperationContext parent, TridentTuple.Factory factory) {
- this(parent._topoContext, factory);
+ this(parent.topoContext, factory);
}
public ProjectionFactory makeProjectionFactory(Fields fields) {
- return new ProjectionFactory(_factory, fields);
+ return new ProjectionFactory(factory, fields);
}
public int numPartitions() {
- return _topoContext.getComponentTasks(_topoContext.getThisComponentId()).size();
+ return topoContext.getComponentTasks(topoContext.getThisComponentId()).size();
}
public int getPartitionIndex() {
- return _topoContext.getThisTaskIndex();
+ return topoContext.getThisTaskIndex();
}
@Override
public <T extends IMetric> T registerMetric(String name, T metric, int timeBucketSizeInSecs) {
- return _topoContext.registerMetric(name, metric, timeBucketSizeInSecs);
+ return topoContext.registerMetric(name, metric, timeBucketSizeInSecs);
}
@Override
public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) {
- return _topoContext.registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs);
+ return topoContext.registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs);
}
@Override
public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) {
- return _topoContext.registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs);
+ return topoContext.registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs);
}
@Override
public Timer registerTimer(String name) {
- return _topoContext.registerTimer(name);
+ return topoContext.registerTimer(name);
}
@Override
public Histogram registerHistogram(String name) {
- return _topoContext.registerHistogram(name);
+ return topoContext.registerHistogram(name);
}
@Override
public Meter registerMeter(String name) {
- return _topoContext.registerMeter(name);
+ return topoContext.registerMeter(name);
}
@Override
public Counter registerCounter(String name) {
- return _topoContext.registerCounter(name);
+ return topoContext.registerCounter(name);
}
@Override
public <T> Gauge<T> registerGauge(String name, Gauge<T> gauge) {
- return _topoContext.registerGauge(name, gauge);
+ return topoContext.registerGauge(name, gauge);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java
index 7b52b3d..4e78ae8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Debug.java
@@ -22,7 +22,7 @@
* Filter for debugging purposes. The `isKeep()` method simply prints the tuple to `System.out` and returns `true`.
*/
public class Debug extends BaseFilter {
- private final Logger LOG = LoggerFactory.getLogger(Debug.class);
+ private static final Logger LOG = LoggerFactory.getLogger(Debug.class);
private final String name;
private boolean useLogger;
@@ -38,8 +38,6 @@
/**
* Creates a `Debug` filter with a string identifier.
- *
- * @param name
*/
public Debug(String name) {
this.name = "DEBUG(" + name + "): ";
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java
index 69fb40c..a2b0992 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/FirstN.java
@@ -24,11 +24,11 @@
/**
- * An {@link org.apache.storm.trident.operation.Assembly} implementation
+ * An {@link org.apache.storm.trident.operation.Assembly} implementation.
*/
public class FirstN implements Assembly {
- Aggregator _agg;
+ Aggregator agg;
public FirstN(int n, String sortField) {
this(n, sortField, false);
@@ -36,25 +36,25 @@
public FirstN(int n, String sortField, boolean reverse) {
if (sortField != null) {
- _agg = new FirstNSortedAgg(n, sortField, reverse);
+ agg = new FirstNSortedAgg(n, sortField, reverse);
} else {
- _agg = new FirstNAgg(n);
+ agg = new FirstNAgg(n);
}
}
@Override
public Stream apply(Stream input) {
Fields outputFields = input.getOutputFields();
- return input.partitionAggregate(outputFields, _agg, outputFields)
+ return input.partitionAggregate(outputFields, agg, outputFields)
.global()
- .partitionAggregate(outputFields, _agg, outputFields);
+ .partitionAggregate(outputFields, agg, outputFields);
}
public static class FirstNAgg extends BaseAggregator<FirstNAgg.State> {
- int _n;
+ int number;
public FirstNAgg(int n) {
- _n = n;
+ number = n;
}
@Override
@@ -64,7 +64,7 @@
@Override
public void aggregate(State val, TridentTuple tuple, TridentCollector collector) {
- if (val.emitted < _n) {
+ if (val.emitted < number) {
collector.emit(tuple);
val.emitted++;
}
@@ -82,25 +82,25 @@
public static class FirstNSortedAgg extends BaseAggregator<PriorityQueue> {
- int _n;
- String _sortField;
- boolean _reverse;
+ int number;
+ String sortField;
+ boolean reverse;
public FirstNSortedAgg(int n, String sortField, boolean reverse) {
- _n = n;
- _sortField = sortField;
- _reverse = reverse;
+ number = n;
+ this.sortField = sortField;
+ this.reverse = reverse;
}
@Override
public PriorityQueue init(Object batchId, TridentCollector collector) {
- return new PriorityQueue(_n, new Comparator<TridentTuple>() {
+ return new PriorityQueue(number, new Comparator<TridentTuple>() {
@Override
public int compare(TridentTuple t1, TridentTuple t2) {
- Comparable c1 = (Comparable) t1.getValueByField(_sortField);
- Comparable c2 = (Comparable) t2.getValueByField(_sortField);
+ Comparable c1 = (Comparable) t1.getValueByField(sortField);
+ Comparable c2 = (Comparable) t2.getValueByField(sortField);
int ret = c1.compareTo(c2);
- if (_reverse) {
+ if (reverse) {
ret *= -1;
}
return ret;
@@ -116,7 +116,7 @@
@Override
public void complete(PriorityQueue val, TridentCollector collector) {
int total = val.size();
- for (int i = 0; i < _n && i < total; i++) {
+ for (int i = 0; i < number && i < total; i++) {
TridentTuple t = (TridentTuple) val.remove();
collector.emit(t);
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java
index 2c315be..b60d30b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MaxWithComparator.java
@@ -37,8 +37,8 @@
@Override
public String toString() {
- return "MaxWithComparator{" +
- "comparator=" + comparator +
- '}';
+ return "MaxWithComparator{"
+ + "comparator=" + comparator
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java
index 7c94f54..1f9eb87 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/MinWithComparator.java
@@ -37,8 +37,8 @@
@Override
public String toString() {
- return "MinWithComparator{" +
- "comparator=" + comparator +
- '}';
+ return "MinWithComparator{"
+ + "comparator=" + comparator
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java
index 8765c32..a2f1380 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/builtin/Negate.java
@@ -20,44 +20,44 @@
/**
* A `Filter` implementation that inverts another delegate `Filter`.
*
- * The `Negate.isKeep()` method simply returns the opposite of the delegate's `isKeep()` method:
+ * <p>The `Negate.isKeep()` method simply returns the opposite of the delegate's `isKeep()` method:
*
- * ```java public boolean isKeep(TridentTuple tuple) { return !this.delegate.isKeep(tuple); } ```
+ * <p>```java public boolean isKeep(TridentTuple tuple) { return !this.delegate.isKeep(tuple); } ```
*
- * The `Negate` filter is useful for dividing a Stream in two based on some boolean condition.
+ * <p>The `Negate` filter is useful for dividing a Stream in two based on some boolean condition.
*
- * Suppose we had a Stream named `userStream` containing information about users, and a custom `Filter` implementation,
+ * <p>Suppose we had a Stream named `userStream` containing information about users, and a custom `Filter` implementation,
* `RegisteredUserFilter` that filtered out unregistered users. We could divide the `userStream` Stream into two separate Streams -- one for
* registered users, and one for unregistered users -- by doing the following:
*
- * ```java Stream userStream = ...
+ * <p>```java Stream userStream = ...
*
- * Filter registeredFilter = new ResisteredUserFilter(); Filter unregisteredFilter = new Negate(registeredFilter);
+ * <p>Filter registeredFilter = new ResisteredUserFilter(); Filter unregisteredFilter = new Negate(registeredFilter);
*
- * Stream registeredUserStream = userStream.each(userStream.getOutputFields(), registeredFilter); Stream unregisteredUserStream =
+ * <p>Stream registeredUserStream = userStream.each(userStream.getOutputFields(), registeredFilter); Stream unregisteredUserStream =
* userStream.each(userStream.getOutputFields(), unregisteredFilter); ```
*/
public class Negate implements Filter {
- Filter _delegate;
+ Filter delegate;
public Negate(Filter delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
@Override
public boolean isKeep(TridentTuple tuple) {
- return !_delegate.isKeep(tuple);
+ return !delegate.isKeep(tuple);
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _delegate.prepare(conf, context);
+ delegate.prepare(conf, context);
}
@Override
public void cleanup() {
- _delegate.cleanup();
+ delegate.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CaptureCollector.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CaptureCollector.java
index 1c4f10b..9f9605c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CaptureCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CaptureCollector.java
@@ -19,10 +19,10 @@
public class CaptureCollector implements TridentCollector {
public List<List<Object>> captured = new ArrayList();
- TridentCollector _coll;
+ TridentCollector coll;
public void setCollector(TridentCollector coll) {
- _coll = coll;
+ this.coll = coll;
}
@Override
@@ -32,11 +32,11 @@
@Override
public void flush() {
- _coll.flush();
+ coll.flush();
}
@Override
public void reportError(Throwable t) {
- _coll.reportError(t);
+ coll.reportError(t);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ChainedAggregatorImpl.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ChainedAggregatorImpl.java
index 6a2ce19..ce45291 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ChainedAggregatorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ChainedAggregatorImpl.java
@@ -24,35 +24,35 @@
import org.apache.storm.tuple.Fields;
public class ChainedAggregatorImpl implements Aggregator<ChainedResult> {
- Aggregator[] _aggs;
- ProjectionFactory[] _inputFactories;
- ComboList.Factory _fact;
- Fields[] _inputFields;
+ Aggregator[] aggs;
+ ProjectionFactory[] inputFactories;
+ ComboList.Factory fact;
+ Fields[] inputFields;
public ChainedAggregatorImpl(Aggregator[] aggs, Fields[] inputFields, ComboList.Factory fact) {
- _aggs = aggs;
- _inputFields = inputFields;
- _fact = fact;
- if (_aggs.length != _inputFields.length) {
+ this.aggs = aggs;
+ this.inputFields = inputFields;
+ this.fact = fact;
+ if (this.aggs.length != this.inputFields.length) {
throw new IllegalArgumentException("Require input fields for each aggregator");
}
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _inputFactories = new ProjectionFactory[_inputFields.length];
- for (int i = 0; i < _inputFields.length; i++) {
- _inputFactories[i] = context.makeProjectionFactory(_inputFields[i]);
- _aggs[i].prepare(conf, new TridentOperationContext(context, _inputFactories[i]));
+ inputFactories = new ProjectionFactory[inputFields.length];
+ for (int i = 0; i < inputFields.length; i++) {
+ inputFactories[i] = context.makeProjectionFactory(inputFields[i]);
+ aggs[i].prepare(conf, new TridentOperationContext(context, inputFactories[i]));
}
}
@Override
public ChainedResult init(Object batchId, TridentCollector collector) {
- ChainedResult initted = new ChainedResult(collector, _aggs.length);
- for (int i = 0; i < _aggs.length; i++) {
- initted.objs[i] = _aggs[i].init(batchId, initted.collectors[i]);
+ ChainedResult initted = new ChainedResult(collector, aggs.length);
+ for (int i = 0; i < aggs.length; i++) {
+ initted.objs[i] = aggs[i].init(batchId, initted.collectors[i]);
}
return initted;
}
@@ -60,19 +60,19 @@
@Override
public void aggregate(ChainedResult val, TridentTuple tuple, TridentCollector collector) {
val.setFollowThroughCollector(collector);
- for (int i = 0; i < _aggs.length; i++) {
- TridentTuple projected = _inputFactories[i].create((TridentTupleView) tuple);
- _aggs[i].aggregate(val.objs[i], projected, val.collectors[i]);
+ for (int i = 0; i < aggs.length; i++) {
+ TridentTuple projected = inputFactories[i].create((TridentTupleView) tuple);
+ aggs[i].aggregate(val.objs[i], projected, val.collectors[i]);
}
}
@Override
public void complete(ChainedResult val, TridentCollector collector) {
val.setFollowThroughCollector(collector);
- for (int i = 0; i < _aggs.length; i++) {
- _aggs[i].complete(val.objs[i], val.collectors[i]);
+ for (int i = 0; i < aggs.length; i++) {
+ aggs[i].complete(val.objs[i], val.collectors[i]);
}
- if (_aggs.length > 1) { // otherwise, tuples were emitted directly
+ if (aggs.length > 1) { // otherwise, tuples were emitted directly
int[] indices = new int[val.collectors.length];
for (int i = 0; i < indices.length; i++) {
indices[i] = 0;
@@ -80,12 +80,12 @@
boolean keepGoing = true;
//emit cross-join of all emitted tuples
while (keepGoing) {
- List[] combined = new List[_aggs.length];
- for (int i = 0; i < _aggs.length; i++) {
+ List[] combined = new List[aggs.length];
+ for (int i = 0; i < aggs.length; i++) {
CaptureCollector capturer = (CaptureCollector) val.collectors[i];
combined[i] = capturer.captured.get(indices[i]);
}
- collector.emit(_fact.create(combined));
+ collector.emit(fact.create(combined));
keepGoing = increment(val.collectors, indices, indices.length - 1);
}
}
@@ -107,7 +107,7 @@
@Override
public void cleanup() {
- for (Aggregator a : _aggs) {
+ for (Aggregator a : aggs) {
a.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggStateUpdater.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggStateUpdater.java
index 00d366a..44e4f03 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggStateUpdater.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggStateUpdater.java
@@ -24,10 +24,10 @@
import org.apache.storm.tuple.Values;
public class CombinerAggStateUpdater implements StateUpdater<Snapshottable> {
- CombinerAggregator _agg;
+ CombinerAggregator agg;
public CombinerAggStateUpdater(CombinerAggregator agg) {
- _agg = agg;
+ this.agg = agg;
}
@@ -36,7 +36,7 @@
if (tuples.size() != 1) {
throw new IllegalArgumentException("Combiner state updater should receive a single tuple. Received: " + tuples.toString());
}
- Object newVal = state.update(new CombinerValueUpdater(_agg, tuples.get(0).getValue(0)));
+ Object newVal = state.update(new CombinerValueUpdater(agg, tuples.get(0).getValue(0)));
collector.emit(new Values(newVal));
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java
index 7525ea5..4427263 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java
@@ -21,10 +21,10 @@
import org.apache.storm.tuple.Values;
public class CombinerAggregatorCombineImpl implements Aggregator<Result> {
- CombinerAggregator _agg;
+ CombinerAggregator agg;
public CombinerAggregatorCombineImpl(CombinerAggregator agg) {
- _agg = agg;
+ this.agg = agg;
}
@Override
@@ -35,7 +35,7 @@
@Override
public Result init(Object batchId, TridentCollector collector) {
Result ret = new Result();
- ret.obj = _agg.zero();
+ ret.obj = agg.zero();
return ret;
}
@@ -45,7 +45,7 @@
if (val.obj == null) {
val.obj = v;
} else {
- val.obj = _agg.combine(val.obj, v);
+ val.obj = agg.combine(val.obj, v);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorInitImpl.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorInitImpl.java
index b7f5715..085797a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorInitImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/CombinerAggregatorInitImpl.java
@@ -22,15 +22,15 @@
public class CombinerAggregatorInitImpl implements Function {
- CombinerAggregator _agg;
+ CombinerAggregator agg;
public CombinerAggregatorInitImpl(CombinerAggregator agg) {
- _agg = agg;
+ this.agg = agg;
}
@Override
public void execute(TridentTuple tuple, TridentCollector collector) {
- collector.emit(new Values(_agg.init(tuple)));
+ collector.emit(new Values(agg.init(tuple)));
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/FilterExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/FilterExecutor.java
index 4c04f6d..09774bb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/FilterExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/FilterExecutor.java
@@ -22,27 +22,27 @@
// works by emitting null to the collector. since the planner knows this is an ADD node with
// no new output fields, it just passes the tuple forward
public class FilterExecutor implements Function {
- Filter _filter;
+ Filter filter;
public FilterExecutor(Filter filter) {
- _filter = filter;
+ this.filter = filter;
}
@Override
public void execute(TridentTuple tuple, TridentCollector collector) {
- if (_filter.isKeep(tuple)) {
+ if (filter.isKeep(tuple)) {
collector.emit(null);
}
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _filter.prepare(conf, context);
+ filter.prepare(conf, context);
}
@Override
public void cleanup() {
- _filter.cleanup();
+ filter.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupCollector.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupCollector.java
index 34d61e2..f70e488 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupCollector.java
@@ -19,12 +19,12 @@
public class GroupCollector implements TridentCollector {
public List<Object> currGroup;
- ComboList.Factory _factory;
- TridentCollector _collector;
+ ComboList.Factory factory;
+ TridentCollector collector;
public GroupCollector(TridentCollector collector, ComboList.Factory factory) {
- _factory = factory;
- _collector = collector;
+ this.factory = factory;
+ this.collector = collector;
}
@Override
@@ -32,17 +32,17 @@
List[] delegates = new List[2];
delegates[0] = currGroup;
delegates[1] = values;
- _collector.emit(_factory.create(delegates));
+ collector.emit(factory.create(delegates));
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void reportError(Throwable t) {
- _collector.reportError(t);
+ collector.reportError(t);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedAggregator.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedAggregator.java
index 54909c6..8598ffb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedAggregator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedAggregator.java
@@ -26,50 +26,50 @@
import org.apache.storm.tuple.Fields;
public class GroupedAggregator implements Aggregator<Object[]> {
- ProjectionFactory _groupFactory;
- ProjectionFactory _inputFactory;
- Aggregator _agg;
- ComboList.Factory _fact;
- Fields _inFields;
- Fields _groupFields;
+ ProjectionFactory groupFactory;
+ ProjectionFactory inputFactory;
+ Aggregator agg;
+ ComboList.Factory fact;
+ Fields inFields;
+ Fields groupFields;
public GroupedAggregator(Aggregator agg, Fields group, Fields input, int outSize) {
- _groupFields = group;
- _inFields = input;
- _agg = agg;
+ groupFields = group;
+ inFields = input;
+ this.agg = agg;
int[] sizes = new int[2];
- sizes[0] = _groupFields.size();
+ sizes[0] = groupFields.size();
sizes[1] = outSize;
- _fact = new ComboList.Factory(sizes);
+ fact = new ComboList.Factory(sizes);
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _inputFactory = context.makeProjectionFactory(_inFields);
- _groupFactory = context.makeProjectionFactory(_groupFields);
- _agg.prepare(conf, new TridentOperationContext(context, _inputFactory));
+ inputFactory = context.makeProjectionFactory(inFields);
+ groupFactory = context.makeProjectionFactory(groupFields);
+ agg.prepare(conf, new TridentOperationContext(context, inputFactory));
}
@Override
public Object[] init(Object batchId, TridentCollector collector) {
- return new Object[]{ new GroupCollector(collector, _fact), new HashMap(), batchId };
+ return new Object[]{ new GroupCollector(collector, fact), new HashMap(), batchId };
}
@Override
public void aggregate(Object[] arr, TridentTuple tuple, TridentCollector collector) {
GroupCollector groupColl = (GroupCollector) arr[0];
Map<List, Object> val = (Map) arr[1];
- TridentTuple group = _groupFactory.create((TridentTupleView) tuple);
- TridentTuple input = _inputFactory.create((TridentTupleView) tuple);
+ TridentTuple group = groupFactory.create((TridentTupleView) tuple);
+ TridentTuple input = inputFactory.create((TridentTupleView) tuple);
Object curr;
if (!val.containsKey(group)) {
- curr = _agg.init(arr[2], groupColl);
+ curr = agg.init(arr[2], groupColl);
val.put((List) group, curr);
} else {
curr = val.get(group);
}
groupColl.currGroup = group;
- _agg.aggregate(curr, input, groupColl);
+ agg.aggregate(curr, input, groupColl);
}
@@ -79,13 +79,13 @@
GroupCollector groupColl = (GroupCollector) arr[0];
for (Entry<List, Object> e : val.entrySet()) {
groupColl.currGroup = e.getKey();
- _agg.complete(e.getValue(), groupColl);
+ agg.complete(e.getValue(), groupColl);
}
}
@Override
public void cleanup() {
- _agg.cleanup();
+ agg.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedMultiReducerExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedMultiReducerExecutor.java
index 34a5f6e..f37c462 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedMultiReducerExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/GroupedMultiReducerExecutor.java
@@ -26,28 +26,28 @@
public class GroupedMultiReducerExecutor implements MultiReducer<Map<TridentTuple, Object>> {
- GroupedMultiReducer _reducer;
- List<Fields> _groupFields;
- List<Fields> _inputFields;
- List<ProjectionFactory> _groupFactories = new ArrayList<ProjectionFactory>();
- List<ProjectionFactory> _inputFactories = new ArrayList<ProjectionFactory>();
+ GroupedMultiReducer reducer;
+ List<Fields> groupFields;
+ List<Fields> inputFields;
+ List<ProjectionFactory> groupFactories = new ArrayList<ProjectionFactory>();
+ List<ProjectionFactory> inputFactories = new ArrayList<ProjectionFactory>();
public GroupedMultiReducerExecutor(GroupedMultiReducer reducer, List<Fields> groupFields, List<Fields> inputFields) {
if (inputFields.size() != groupFields.size()) {
throw new IllegalArgumentException("Multireducer groupFields and inputFields must be the same size");
}
- _groupFields = groupFields;
- _inputFields = inputFields;
- _reducer = reducer;
+ this.groupFields = groupFields;
+ this.inputFields = inputFields;
+ this.reducer = reducer;
}
@Override
public void prepare(Map<String, Object> conf, TridentMultiReducerContext context) {
- for (int i = 0; i < _groupFields.size(); i++) {
- _groupFactories.add(context.makeProjectionFactory(i, _groupFields.get(i)));
- _inputFactories.add(context.makeProjectionFactory(i, _inputFields.get(i)));
+ for (int i = 0; i < groupFields.size(); i++) {
+ groupFactories.add(context.makeProjectionFactory(i, groupFields.get(i)));
+ inputFactories.add(context.makeProjectionFactory(i, inputFields.get(i)));
}
- _reducer.prepare(conf, new TridentMultiReducerContext((List) _inputFactories));
+ reducer.prepare(conf, new TridentMultiReducerContext((List) inputFactories));
}
@Override
@@ -57,20 +57,20 @@
@Override
public void execute(Map<TridentTuple, Object> state, int streamIndex, TridentTuple full, TridentCollector collector) {
- ProjectionFactory groupFactory = _groupFactories.get(streamIndex);
- ProjectionFactory inputFactory = _inputFactories.get(streamIndex);
+ ProjectionFactory groupFactory = groupFactories.get(streamIndex);
+ ProjectionFactory inputFactory = inputFactories.get(streamIndex);
TridentTuple group = groupFactory.create(full);
TridentTuple input = inputFactory.create(full);
Object curr;
if (!state.containsKey(group)) {
- curr = _reducer.init(collector, group);
+ curr = reducer.init(collector, group);
state.put(group, curr);
} else {
curr = state.get(group);
}
- _reducer.execute(curr, streamIndex, group, input, collector);
+ reducer.execute(curr, streamIndex, group, input, collector);
}
@Override
@@ -78,13 +78,13 @@
for (Map.Entry e : state.entrySet()) {
TridentTuple group = (TridentTuple) e.getKey();
Object val = e.getValue();
- _reducer.complete(val, group, collector);
+ reducer.complete(val, group, collector);
}
}
@Override
public void cleanup() {
- _reducer.cleanup();
+ reducer.cleanup();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/JoinerMultiReducer.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/JoinerMultiReducer.java
index 0d15d0a..6eb33d8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/JoinerMultiReducer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/JoinerMultiReducer.java
@@ -25,31 +25,31 @@
public class JoinerMultiReducer implements GroupedMultiReducer<JoinState> {
- List<JoinType> _types;
- List<Fields> _sideFields;
- int _numGroupFields;
- ComboList.Factory _factory;
+ List<JoinType> types;
+ List<Fields> sideFields;
+ int numGroupFields;
+ ComboList.Factory factory;
public JoinerMultiReducer(List<JoinType> types, int numGroupFields, List<Fields> sides) {
- _types = types;
- _sideFields = sides;
- _numGroupFields = numGroupFields;
+ this.types = types;
+ sideFields = sides;
+ this.numGroupFields = numGroupFields;
}
@Override
public void prepare(Map<String, Object> conf, TridentMultiReducerContext context) {
- int[] sizes = new int[_sideFields.size() + 1];
- sizes[0] = _numGroupFields;
- for (int i = 0; i < _sideFields.size(); i++) {
- sizes[i + 1] = _sideFields.get(i).size();
+ int[] sizes = new int[sideFields.size() + 1];
+ sizes[0] = numGroupFields;
+ for (int i = 0; i < sideFields.size(); i++) {
+ sizes[i + 1] = sideFields.get(i).size();
}
- _factory = new ComboList.Factory(sizes);
+ factory = new ComboList.Factory(sizes);
}
@Override
public JoinState init(TridentCollector collector, TridentTuple group) {
- return new JoinState(_types.size(), group);
+ return new JoinState(types.size(), group);
}
@Override
@@ -72,9 +72,9 @@
List<List>[] sides = state.sides;
boolean wasEmpty = state.numSidesReceived < sides.length;
for (int i = 0; i < sides.length; i++) {
- if (sides[i].isEmpty() && _types.get(i) == JoinType.OUTER) {
+ if (sides[i].isEmpty() && types.get(i) == JoinType.OUTER) {
state.numSidesReceived++;
- sides[i].add(makeNullList(_sideFields.get(i).size()));
+ sides[i].add(makeNullList(sideFields.get(i).size()));
}
}
if (wasEmpty && state.numSidesReceived == sides.length) {
@@ -113,7 +113,7 @@
combined[i + 1] = sides[i].get(indices[i]);
}
}
- collector.emit(_factory.create(combined));
+ collector.emit(factory.create(combined));
keepGoing = increment(sides, indices, indices.length - 1, overrideIndex);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/PreservingFieldsOrderJoinerMultiReducer.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/PreservingFieldsOrderJoinerMultiReducer.java
index 9548a5b..08a45fc 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/PreservingFieldsOrderJoinerMultiReducer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/PreservingFieldsOrderJoinerMultiReducer.java
@@ -24,37 +24,37 @@
import org.apache.storm.tuple.Fields;
public class PreservingFieldsOrderJoinerMultiReducer implements GroupedMultiReducer<JoinState> {
- List<JoinType> _types;
- List<Fields> _sideFields;
- List<Fields> _joiningFields;
- List<Fields> _originFields;
- int _numGroupFields;
- ComboList.Factory _factory;
+ List<JoinType> types;
+ List<Fields> sideFields;
+ List<Fields> joiningFields;
+ List<Fields> originFields;
+ int numGroupFields;
+ ComboList.Factory factory;
public PreservingFieldsOrderJoinerMultiReducer(List<JoinType> types, int numGroupFields, List<Fields> origins,
List<Fields> joins, List<Fields> sides) {
- _types = types;
- _originFields = origins;
- _joiningFields = joins;
- _sideFields = sides;
+ this.types = types;
+ originFields = origins;
+ joiningFields = joins;
+ sideFields = sides;
// we already checked this
- _numGroupFields = numGroupFields;
+ this.numGroupFields = numGroupFields;
}
@Override
public void prepare(Map<String, Object> conf, TridentMultiReducerContext context) {
- int[] sizes = new int[_originFields.size()];
- for (int i = 0; i < _originFields.size(); i++) {
- sizes[i] = _originFields.get(i).size();
+ int[] sizes = new int[originFields.size()];
+ for (int i = 0; i < originFields.size(); i++) {
+ sizes[i] = originFields.get(i).size();
}
- _factory = new ComboList.Factory(sizes);
+ factory = new ComboList.Factory(sizes);
}
@Override
public JoinState init(TridentCollector collector, TridentTuple group) {
- return new JoinState(_types.size(), group);
+ return new JoinState(types.size(), group);
}
@Override
@@ -77,7 +77,7 @@
List<List>[] sides = state.sides;
boolean wasEmpty = state.numSidesReceived < sides.length;
for (int i = 0; i < sides.length; i++) {
- if (sides[i].isEmpty() && _types.get(i) == JoinType.OUTER) {
+ if (sides[i].isEmpty() && types.get(i) == JoinType.OUTER) {
state.numSidesReceived++;
sides[i].add(null);
}
@@ -115,7 +115,7 @@
List<Object> values = buildValuesForStream(state, overrideIndex, overrideTuple, sides, indices, combined, i);
combined[i] = values;
}
- collector.emit(_factory.create(combined));
+ collector.emit(factory.create(combined));
keepGoing = increment(sides, indices, indices.length - 1, overrideIndex);
}
}
@@ -129,16 +129,16 @@
sideValues = sides[streamIdx].get(indices[streamIdx]);
}
- Fields originFields = _originFields.get(streamIdx);
+ Fields originFields = this.originFields.get(streamIdx);
if (sideValues == null) {
return makeNullList(originFields.size());
} else {
List<Object> ret = new ArrayList<>(originFields.size());
- Fields sideFields = _sideFields.get(streamIdx);
- Fields joinFields = _joiningFields.get(streamIdx);
+ Fields sideFields = this.sideFields.get(streamIdx);
+ Fields joinFields = joiningFields.get(streamIdx);
int sideIdx = 0;
for (String field : originFields) {
- // assuming _sideFields are preserving its order
+ // assuming sideFields are preserving its order
if (sideFields.contains(field)) {
ret.add(sideValues.get(sideIdx++));
} else {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggStateUpdater.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggStateUpdater.java
index f909609..84caa75 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggStateUpdater.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggStateUpdater.java
@@ -24,16 +24,16 @@
import org.apache.storm.tuple.Values;
public class ReducerAggStateUpdater implements StateUpdater<Snapshottable> {
- ReducerAggregator _agg;
+ ReducerAggregator agg;
public ReducerAggStateUpdater(ReducerAggregator agg) {
- _agg = agg;
+ this.agg = agg;
}
@Override
public void updateState(Snapshottable state, List<TridentTuple> tuples, TridentCollector collector) {
- Object newVal = state.update(new ReducerValueUpdater(_agg, tuples));
+ Object newVal = state.update(new ReducerValueUpdater(agg, tuples));
collector.emit(new Values(newVal));
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggregatorImpl.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggregatorImpl.java
index 0b6a507..bf208cb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggregatorImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/ReducerAggregatorImpl.java
@@ -21,10 +21,10 @@
import org.apache.storm.tuple.Values;
public class ReducerAggregatorImpl implements Aggregator<Result> {
- ReducerAggregator _agg;
+ ReducerAggregator agg;
public ReducerAggregatorImpl(ReducerAggregator agg) {
- _agg = agg;
+ this.agg = agg;
}
@Override
@@ -35,13 +35,13 @@
@Override
public Result init(Object batchId, TridentCollector collector) {
Result ret = new Result();
- ret.obj = _agg.init();
+ ret.obj = agg.init();
return ret;
}
@Override
public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) {
- val.obj = _agg.reduce(val.obj, tuple);
+ val.obj = agg.reduce(val.obj, tuple);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/SingleEmitAggregator.java b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/SingleEmitAggregator.java
index 5f9855b..0e50439 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/operation/impl/SingleEmitAggregator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/operation/impl/SingleEmitAggregator.java
@@ -22,14 +22,14 @@
public class SingleEmitAggregator implements Aggregator<SingleEmitState> {
- Aggregator _agg;
- BatchToPartition _batchToPartition;
+ Aggregator agg;
+ BatchToPartition batchToPartition;
int myPartitionIndex;
int totalPartitions;
public SingleEmitAggregator(Aggregator agg, BatchToPartition batchToPartition) {
- _agg = agg;
- _batchToPartition = batchToPartition;
+ this.agg = agg;
+ this.batchToPartition = batchToPartition;
}
@@ -41,34 +41,34 @@
@Override
public void aggregate(SingleEmitState val, TridentTuple tuple, TridentCollector collector) {
if (!val.received) {
- val.state = _agg.init(val.batchId, collector);
+ val.state = agg.init(val.batchId, collector);
val.received = true;
}
- _agg.aggregate(val.state, tuple, collector);
+ agg.aggregate(val.state, tuple, collector);
}
@Override
public void complete(SingleEmitState val, TridentCollector collector) {
if (!val.received) {
- if (this.myPartitionIndex == _batchToPartition.partitionIndex(val.batchId, this.totalPartitions)) {
- val.state = _agg.init(val.batchId, collector);
- _agg.complete(val.state, collector);
+ if (this.myPartitionIndex == batchToPartition.partitionIndex(val.batchId, this.totalPartitions)) {
+ val.state = agg.init(val.batchId, collector);
+ agg.complete(val.state, collector);
}
} else {
- _agg.complete(val.state, collector);
+ agg.complete(val.state, collector);
}
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _agg.prepare(conf, context);
+ agg.prepare(conf, context);
this.myPartitionIndex = context.getPartitionIndex();
this.totalPartitions = context.numPartitions();
}
@Override
public void cleanup() {
- _agg.cleanup();
+ agg.cleanup();
}
public static interface BatchToPartition extends Serializable {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/partition/IdentityGrouping.java b/storm-client/src/jvm/org/apache/storm/trident/partition/IdentityGrouping.java
index b4092ac..9617ea8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/partition/IdentityGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/partition/IdentityGrouping.java
@@ -24,7 +24,7 @@
public class IdentityGrouping implements CustomStreamGrouping {
- final Map<Integer, List<Integer>> _precomputed = new HashMap<>();
+ final Map<Integer, List<Integer>> precomputed = new HashMap<>();
@Override
public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> tasks) {
@@ -38,13 +38,13 @@
for (int i = 0; i < sourceTasks.size(); i++) {
int s = sourceTasks.get(i);
int t = tasks.get(i);
- _precomputed.put(s, Arrays.asList(t));
+ precomputed.put(s, Arrays.asList(t));
}
}
@Override
public List<Integer> chooseTasks(int task, List<Object> values) {
- List<Integer> ret = _precomputed.get(task);
+ List<Integer> ret = precomputed.get(task);
if (ret == null) {
throw new RuntimeException("Tuple emitted by task that's not part of this component. Should be impossible");
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/partition/IndexHashGrouping.java b/storm-client/src/jvm/org/apache/storm/trident/partition/IndexHashGrouping.java
index 35ad13f..7a2cfc1 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/partition/IndexHashGrouping.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/partition/IndexHashGrouping.java
@@ -20,11 +20,11 @@
import org.apache.storm.utils.Utils;
public class IndexHashGrouping implements CustomStreamGrouping {
- int _index;
- List<Integer> _targets;
+ int index;
+ List<Integer> targets;
public IndexHashGrouping(int index) {
- _index = index;
+ this.index = index;
}
public static int objectToIndex(Object val, int numPartitions) {
@@ -36,13 +36,13 @@
@Override
public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
- _targets = targetTasks;
+ targets = targetTasks;
}
@Override
public List<Integer> chooseTasks(int fromTask, List<Object> values) {
- int i = objectToIndex(values.get(_index), _targets.size());
- return Arrays.asList(_targets.get(i));
+ int i = objectToIndex(values.get(index), targets.size());
+ return Arrays.asList(targets.get(i));
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/BridgeReceiver.java b/storm-client/src/jvm/org/apache/storm/trident/planner/BridgeReceiver.java
index e5c1fb1..601a64b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/BridgeReceiver.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/BridgeReceiver.java
@@ -19,19 +19,19 @@
public class BridgeReceiver implements TupleReceiver {
- BatchOutputCollector _collector;
+ BatchOutputCollector collector;
public BridgeReceiver(BatchOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
public void execute(ProcessorContext context, String streamId, TridentTuple tuple) {
- _collector.emit(streamId, new ConsList(context.batchId, tuple));
+ collector.emit(streamId, new ConsList(context.batchId, tuple));
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/SubtopologyBolt.java b/storm-client/src/jvm/org/apache/storm/trident/planner/SubtopologyBolt.java
index f375ddf..6afd99c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/SubtopologyBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/SubtopologyBolt.java
@@ -45,35 +45,35 @@
public class SubtopologyBolt implements ITridentBatchBolt {
private static final long serialVersionUID = 1475508603138688412L;
@SuppressWarnings("rawtypes")
- final DirectedGraph<Node, IndexedEdge> _graph;
- final Set<Node> _nodes;
- final Map<String, InitialReceiver> _roots = new HashMap<>();
- final Map<Node, Factory> _outputFactories = new HashMap<>();
- final Map<String, List<TridentProcessor>> _myTopologicallyOrdered = new HashMap<>();
- final Map<Node, String> _batchGroups;
+ final DirectedGraph<Node, IndexedEdge> graph;
+ final Set<Node> nodes;
+ final Map<String, InitialReceiver> roots = new HashMap<>();
+ final Map<Node, Factory> outputFactories = new HashMap<>();
+ final Map<String, List<TridentProcessor>> myTopologicallyOrdered = new HashMap<>();
+ final Map<Node, String> batchGroups;
//given processornodes and static state nodes
@SuppressWarnings({ "unchecked", "rawtypes" })
public SubtopologyBolt(DefaultDirectedGraph<Node, IndexedEdge> graph, Set<Node> nodes, Map<Node, String> batchGroups) {
- _nodes = nodes;
- _graph = (DirectedGraph<Node, IndexedEdge>) graph.clone();
- _batchGroups = copyAndOnlyKeep(batchGroups, nodes);
+ this.nodes = nodes;
+ this.graph = (DirectedGraph<Node, IndexedEdge>) graph.clone();
+ this.batchGroups = copyAndOnlyKeep(batchGroups, nodes);
//Remove the unneeded entries from the graph
//We want to keep all of our nodes, and the nodes that they are connected directly to (parents and children).
Set<Node> nodesToKeep = new HashSet<>();
- for (IndexedEdge edge : _graph.edgeSet()) {
- Node s = _graph.getEdgeSource(edge);
- Node t = _graph.getEdgeTarget(edge);
- if (_nodes.contains(s) || _nodes.contains(t)) {
+ for (IndexedEdge edge : this.graph.edgeSet()) {
+ Node s = this.graph.getEdgeSource(edge);
+ Node t = this.graph.getEdgeTarget(edge);
+ if (this.nodes.contains(s) || this.nodes.contains(t)) {
nodesToKeep.add(s);
nodesToKeep.add(t);
}
}
- Set<Node> nodesToRemove = new HashSet<>(_graph.vertexSet());
+ Set<Node> nodesToRemove = new HashSet<>(this.graph.vertexSet());
nodesToRemove.removeAll(nodesToKeep);
- _graph.removeAllVertices(nodesToRemove);
+ this.graph.removeAllVertices(nodesToRemove);
}
private static Map<Node, String> copyAndOnlyKeep(Map<Node, String> batchGroups, Set<Node> nodes) {
@@ -89,42 +89,42 @@
@Override
public void prepare(Map<String, Object> conf, TopologyContext context, BatchOutputCollector batchCollector) {
int thisComponentNumTasks = context.getComponentTasks(context.getThisComponentId()).size();
- for (Node n : _nodes) {
+ for (Node n : nodes) {
if (n.stateInfo != null) {
State s = n.stateInfo.spec.stateFactory.makeState(conf, context, context.getThisTaskIndex(), thisComponentNumTasks);
context.setTaskData(n.stateInfo.id, s);
}
}
- DirectedSubgraph<Node, ?> subgraph = new DirectedSubgraph<>(_graph, _nodes, null);
+ DirectedSubgraph<Node, ?> subgraph = new DirectedSubgraph<>(graph, nodes, null);
TopologicalOrderIterator<Node, ?> it = new TopologicalOrderIterator<>(subgraph);
int stateIndex = 0;
while (it.hasNext()) {
Node n = it.next();
if (n instanceof ProcessorNode) {
ProcessorNode pn = (ProcessorNode) n;
- String batchGroup = _batchGroups.get(n);
- if (!_myTopologicallyOrdered.containsKey(batchGroup)) {
- _myTopologicallyOrdered.put(batchGroup, new ArrayList<>());
+ String batchGroup = batchGroups.get(n);
+ if (!myTopologicallyOrdered.containsKey(batchGroup)) {
+ myTopologicallyOrdered.put(batchGroup, new ArrayList<>());
}
- _myTopologicallyOrdered.get(batchGroup).add(pn.processor);
+ myTopologicallyOrdered.get(batchGroup).add(pn.processor);
List<String> parentStreams = new ArrayList<>();
List<Factory> parentFactories = new ArrayList<>();
- for (Node p : TridentUtils.getParents(_graph, n)) {
+ for (Node p : TridentUtils.getParents(graph, n)) {
parentStreams.add(p.streamId);
- if (_nodes.contains(p)) {
- parentFactories.add(_outputFactories.get(p));
+ if (nodes.contains(p)) {
+ parentFactories.add(outputFactories.get(p));
} else {
- if (!_roots.containsKey(p.streamId)) {
- _roots.put(p.streamId, new InitialReceiver(p.streamId, getSourceOutputFields(context, p.streamId)));
+ if (!roots.containsKey(p.streamId)) {
+ roots.put(p.streamId, new InitialReceiver(p.streamId, getSourceOutputFields(context, p.streamId)));
}
- _roots.get(p.streamId).addReceiver(pn.processor);
- parentFactories.add(_roots.get(p.streamId).getOutputFactory());
+ roots.get(p.streamId).addReceiver(pn.processor);
+ parentFactories.add(roots.get(p.streamId).getOutputFactory());
}
}
List<TupleReceiver> targets = new ArrayList<>();
boolean outgoingNode = false;
- for (Node cn : TridentUtils.getChildren(_graph, n)) {
- if (_nodes.contains(cn)) {
+ for (Node cn : TridentUtils.getChildren(graph, n)) {
+ if (nodes.contains(cn)) {
targets.add(((ProcessorNode) cn).processor);
} else {
outgoingNode = true;
@@ -144,7 +144,7 @@
batchCollector
);
pn.processor.prepare(conf, context, triContext);
- _outputFactories.put(n, pn.processor.getOutputFactory());
+ outputFactories.put(n, pn.processor.getOutputFactory());
}
stateIndex++;
}
@@ -162,7 +162,7 @@
@Override
public void execute(BatchInfo batchInfo, Tuple tuple) {
String sourceStream = tuple.getSourceStreamId();
- InitialReceiver ir = _roots.get(sourceStream);
+ InitialReceiver ir = roots.get(sourceStream);
if (ir == null) {
throw new RuntimeException("Received unexpected tuple " + tuple.toString());
}
@@ -171,15 +171,15 @@
@Override
public void finishBatch(BatchInfo batchInfo) {
- for (TridentProcessor p : _myTopologicallyOrdered.get(batchInfo.batchGroup)) {
+ for (TridentProcessor p : myTopologicallyOrdered.get(batchInfo.batchGroup)) {
p.finishBatch((ProcessorContext) batchInfo.state);
}
}
@Override
public Object initBatchState(String batchGroup, Object batchId) {
- ProcessorContext ret = new ProcessorContext(batchId, new Object[_nodes.size()]);
- for (TridentProcessor p : _myTopologicallyOrdered.get(batchGroup)) {
+ ProcessorContext ret = new ProcessorContext(batchId, new Object[nodes.size()]);
+ for (TridentProcessor p : myTopologicallyOrdered.get(batchGroup)) {
p.startBatch(ret);
}
return ret;
@@ -187,8 +187,8 @@
@Override
public void cleanup() {
- for (String bg : _myTopologicallyOrdered.keySet()) {
- for (TridentProcessor p : _myTopologicallyOrdered.get(bg)) {
+ for (String bg : myTopologicallyOrdered.keySet()) {
+ for (TridentProcessor p : myTopologicallyOrdered.get(bg)) {
p.cleanup();
}
}
@@ -196,7 +196,7 @@
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- for (Node n : _nodes) {
+ for (Node n : nodes) {
declarer.declareStream(n.streamId, TridentUtils.fieldsConcat(new Fields("$batchId"), n.allOutputFields));
}
}
@@ -208,32 +208,32 @@
protected static class InitialReceiver {
- List<TridentProcessor> _receivers = new ArrayList<>();
- RootFactory _factory;
- ProjectionFactory _project;
- String _stream;
+ List<TridentProcessor> receivers = new ArrayList<>();
+ RootFactory factory;
+ ProjectionFactory project;
+ String stream;
public InitialReceiver(String stream, Fields allFields) {
- _stream = stream;
- _factory = new RootFactory(allFields);
+ this.stream = stream;
+ factory = new RootFactory(allFields);
List<String> projected = new ArrayList<>(allFields.toList());
projected.remove(0);
- _project = new ProjectionFactory(_factory, new Fields(projected));
+ project = new ProjectionFactory(factory, new Fields(projected));
}
public void receive(ProcessorContext context, Tuple tuple) {
- TridentTuple t = _project.create(_factory.create(tuple));
- for (TridentProcessor r : _receivers) {
- r.execute(context, _stream, t);
+ TridentTuple t = project.create(factory.create(tuple));
+ for (TridentProcessor r : receivers) {
+ r.execute(context, stream, t);
}
}
public void addReceiver(TridentProcessor p) {
- _receivers.add(p);
+ receivers.add(p);
}
public Factory getOutputFactory() {
- return _project;
+ return project;
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AggregateProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AggregateProcessor.java
index 498276e..61476ee 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AggregateProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AggregateProcessor.java
@@ -26,15 +26,15 @@
public class AggregateProcessor implements TridentProcessor {
- Aggregator _agg;
- TridentContext _context;
- FreshCollector _collector;
- Fields _inputFields;
- ProjectionFactory _projection;
+ Aggregator agg;
+ TridentContext context;
+ FreshCollector collector;
+ Fields inputFields;
+ ProjectionFactory projection;
public AggregateProcessor(Fields inputFields, Aggregator agg) {
- _agg = agg;
- _inputFields = inputFields;
+ this.agg = agg;
+ this.inputFields = inputFields;
}
@Override
@@ -43,42 +43,42 @@
if (parents.size() != 1) {
throw new RuntimeException("Aggregate operation can only have one parent");
}
- _context = tridentContext;
- _collector = new FreshCollector(tridentContext);
- _projection = new ProjectionFactory(parents.get(0), _inputFields);
- _agg.prepare(conf, new TridentOperationContext(context, _projection));
+ this.context = tridentContext;
+ collector = new FreshCollector(tridentContext);
+ projection = new ProjectionFactory(parents.get(0), inputFields);
+ agg.prepare(conf, new TridentOperationContext(context, projection));
}
@Override
public void cleanup() {
- _agg.cleanup();
+ agg.cleanup();
}
@Override
public void startBatch(ProcessorContext processorContext) {
- _collector.setContext(processorContext);
- processorContext.state[_context.getStateIndex()] = _agg.init(processorContext.batchId, _collector);
+ collector.setContext(processorContext);
+ processorContext.state[context.getStateIndex()] = agg.init(processorContext.batchId, collector);
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- _collector.setContext(processorContext);
- _agg.aggregate(processorContext.state[_context.getStateIndex()], _projection.create(tuple), _collector);
+ collector.setContext(processorContext);
+ agg.aggregate(processorContext.state[context.getStateIndex()], projection.create(tuple), collector);
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void finishBatch(ProcessorContext processorContext) {
- _collector.setContext(processorContext);
- _agg.complete(processorContext.state[_context.getStateIndex()], _collector);
+ collector.setContext(processorContext);
+ agg.complete(processorContext.state[context.getStateIndex()], collector);
}
@Override
public Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AppendCollector.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AppendCollector.java
index 9e140f6..cde0a5e 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AppendCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/AppendCollector.java
@@ -23,14 +23,14 @@
public class AppendCollector implements TridentCollector {
- OperationOutputFactory _factory;
- TridentContext _triContext;
+ OperationOutputFactory factory;
+ TridentContext triContext;
TridentTuple tuple;
ProcessorContext context;
public AppendCollector(TridentContext context) {
- _triContext = context;
- _factory = new OperationOutputFactory(context.getParentTupleFactories().get(0), context.getSelfOutputFields());
+ triContext = context;
+ factory = new OperationOutputFactory(context.getParentTupleFactories().get(0), context.getSelfOutputFields());
}
public void setContext(ProcessorContext pc, TridentTuple t) {
@@ -40,25 +40,25 @@
@Override
public void emit(List<Object> values) {
- TridentTuple toEmit = _factory.create((TridentTupleView) tuple, values);
- for (TupleReceiver r : _triContext.getReceivers()) {
- r.execute(context, _triContext.getOutStreamId(), toEmit);
+ TridentTuple toEmit = factory.create((TridentTupleView) tuple, values);
+ for (TupleReceiver r : triContext.getReceivers()) {
+ r.execute(context, triContext.getOutStreamId(), toEmit);
}
}
@Override
public void flush() {
- for (TupleReceiver r : _triContext.getReceivers()) {
+ for (TupleReceiver r : triContext.getReceivers()) {
r.flush();
}
}
@Override
public void reportError(Throwable t) {
- _triContext.getDelegateCollector().reportError(t);
+ triContext.getDelegateCollector().reportError(t);
}
public Factory getOutputFactory() {
- return _factory;
+ return factory;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/EachProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/EachProcessor.java
index 746acfe..386df46 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/EachProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/EachProcessor.java
@@ -26,15 +26,15 @@
public class EachProcessor implements TridentProcessor {
- Function _function;
- TridentContext _context;
- AppendCollector _collector;
- Fields _inputFields;
- ProjectionFactory _projection;
+ Function function;
+ TridentContext context;
+ AppendCollector collector;
+ Fields inputFields;
+ ProjectionFactory projection;
public EachProcessor(Fields inputFields, Function function) {
- _function = function;
- _inputFields = inputFields;
+ this.function = function;
+ this.inputFields = inputFields;
}
@Override
@@ -43,26 +43,26 @@
if (parents.size() != 1) {
throw new RuntimeException("Each operation can only have one parent");
}
- _context = tridentContext;
- _collector = new AppendCollector(tridentContext);
- _projection = new ProjectionFactory(parents.get(0), _inputFields);
- _function.prepare(conf, new TridentOperationContext(context, _projection));
+ this.context = tridentContext;
+ collector = new AppendCollector(tridentContext);
+ projection = new ProjectionFactory(parents.get(0), inputFields);
+ function.prepare(conf, new TridentOperationContext(context, projection));
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void cleanup() {
- _function.cleanup();
+ function.cleanup();
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- _collector.setContext(processorContext, tuple);
- _function.execute(_projection.create(tuple), _collector);
+ collector.setContext(processorContext, tuple);
+ function.execute(projection.create(tuple), collector);
}
@@ -76,6 +76,6 @@
@Override
public Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/FreshCollector.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/FreshCollector.java
index cc37df1..21843cc 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/FreshCollector.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/FreshCollector.java
@@ -22,13 +22,13 @@
public class FreshCollector implements TridentCollector {
- FreshOutputFactory _factory;
- TridentContext _triContext;
+ FreshOutputFactory factory;
+ TridentContext triContext;
ProcessorContext context;
public FreshCollector(TridentContext context) {
- _triContext = context;
- _factory = new FreshOutputFactory(context.getSelfOutputFields());
+ triContext = context;
+ factory = new FreshOutputFactory(context.getSelfOutputFields());
}
public void setContext(ProcessorContext pc) {
@@ -37,25 +37,25 @@
@Override
public void emit(List<Object> values) {
- TridentTuple toEmit = _factory.create(values);
- for (TupleReceiver r : _triContext.getReceivers()) {
- r.execute(context, _triContext.getOutStreamId(), toEmit);
+ TridentTuple toEmit = factory.create(values);
+ for (TupleReceiver r : triContext.getReceivers()) {
+ r.execute(context, triContext.getOutStreamId(), toEmit);
}
}
@Override
public void flush() {
- for (TupleReceiver r : _triContext.getReceivers()) {
+ for (TupleReceiver r : triContext.getReceivers()) {
r.flush();
}
}
@Override
public void reportError(Throwable t) {
- _triContext.getDelegateCollector().reportError(t);
+ triContext.getDelegateCollector().reportError(t);
}
public Factory getOutputFactory() {
- return _factory;
+ return factory;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MapProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MapProcessor.java
index b2ed389..2895463 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MapProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MapProcessor.java
@@ -31,15 +31,15 @@
* functions.
*/
public class MapProcessor implements TridentProcessor {
- Function _function;
- TridentContext _context;
- FreshCollector _collector;
- Fields _inputFields;
- TridentTupleView.ProjectionFactory _projection;
+ Function function;
+ TridentContext context;
+ FreshCollector collector;
+ Fields inputFields;
+ TridentTupleView.ProjectionFactory projection;
public MapProcessor(Fields inputFields, Function function) {
- _function = function;
- _inputFields = inputFields;
+ this.function = function;
+ this.inputFields = inputFields;
}
@Override
@@ -48,26 +48,26 @@
if (parents.size() != 1) {
throw new RuntimeException("Map operation can only have one parent");
}
- _context = tridentContext;
- _collector = new FreshCollector(tridentContext);
- _projection = new TridentTupleView.ProjectionFactory(parents.get(0), _inputFields);
- _function.prepare(conf, new TridentOperationContext(context, _projection));
+ this.context = tridentContext;
+ collector = new FreshCollector(tridentContext);
+ projection = new TridentTupleView.ProjectionFactory(parents.get(0), inputFields);
+ function.prepare(conf, new TridentOperationContext(context, projection));
}
@Override
public void cleanup() {
- _function.cleanup();
+ function.cleanup();
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- _collector.setContext(processorContext);
- _function.execute(_projection.create(tuple), _collector);
+ collector.setContext(processorContext);
+ function.execute(projection.create(tuple), collector);
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
@@ -82,6 +82,6 @@
@Override
public TridentTuple.Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MultiReducerProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MultiReducerProcessor.java
index 4e698fd..32b0076 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MultiReducerProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/MultiReducerProcessor.java
@@ -28,66 +28,66 @@
public class MultiReducerProcessor implements TridentProcessor {
- MultiReducer _reducer;
- TridentContext _context;
- Map<String, Integer> _streamToIndex;
- List<Fields> _projectFields;
- ProjectionFactory[] _projectionFactories;
- FreshCollector _collector;
+ MultiReducer reducer;
+ TridentContext context;
+ Map<String, Integer> streamToIndex;
+ List<Fields> projectFields;
+ ProjectionFactory[] projectionFactories;
+ FreshCollector collector;
public MultiReducerProcessor(List<Fields> inputFields, MultiReducer reducer) {
- _reducer = reducer;
- _projectFields = inputFields;
+ this.reducer = reducer;
+ projectFields = inputFields;
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context, TridentContext tridentContext) {
List<Factory> parents = tridentContext.getParentTupleFactories();
- _context = tridentContext;
- _streamToIndex = new HashMap<>();
+ this.context = tridentContext;
+ streamToIndex = new HashMap<>();
List<String> parentStreams = tridentContext.getParentStreams();
for (int i = 0; i < parentStreams.size(); i++) {
- _streamToIndex.put(parentStreams.get(i), i);
+ streamToIndex.put(parentStreams.get(i), i);
}
- _projectionFactories = new ProjectionFactory[_projectFields.size()];
- for (int i = 0; i < _projectFields.size(); i++) {
- _projectionFactories[i] = new ProjectionFactory(parents.get(i), _projectFields.get(i));
+ projectionFactories = new ProjectionFactory[projectFields.size()];
+ for (int i = 0; i < projectFields.size(); i++) {
+ projectionFactories[i] = new ProjectionFactory(parents.get(i), projectFields.get(i));
}
- _collector = new FreshCollector(tridentContext);
- _reducer.prepare(conf, new TridentMultiReducerContext((List) Arrays.asList(_projectionFactories)));
+ collector = new FreshCollector(tridentContext);
+ reducer.prepare(conf, new TridentMultiReducerContext((List) Arrays.asList(projectionFactories)));
}
@Override
public void cleanup() {
- _reducer.cleanup();
+ reducer.cleanup();
}
@Override
public void startBatch(ProcessorContext processorContext) {
- _collector.setContext(processorContext);
- processorContext.state[_context.getStateIndex()] = _reducer.init(_collector);
+ collector.setContext(processorContext);
+ processorContext.state[context.getStateIndex()] = reducer.init(collector);
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- _collector.setContext(processorContext);
- int i = _streamToIndex.get(streamId);
- _reducer.execute(processorContext.state[_context.getStateIndex()], i, _projectionFactories[i].create(tuple), _collector);
+ collector.setContext(processorContext);
+ int i = streamToIndex.get(streamId);
+ reducer.execute(processorContext.state[context.getStateIndex()], i, projectionFactories[i].create(tuple), collector);
}
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void finishBatch(ProcessorContext processorContext) {
- _collector.setContext(processorContext);
- _reducer.complete(processorContext.state[_context.getStateIndex()], _collector);
+ collector.setContext(processorContext);
+ reducer.complete(processorContext.state[context.getStateIndex()], collector);
}
@Override
public Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/PartitionPersistProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/PartitionPersistProcessor.java
index bad9631..a82f39b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/PartitionPersistProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/PartitionPersistProcessor.java
@@ -29,18 +29,18 @@
public class PartitionPersistProcessor implements TridentProcessor {
- StateUpdater _updater;
- State _state;
- String _stateId;
- TridentContext _context;
- Fields _inputFields;
- ProjectionFactory _projection;
- FreshCollector _collector;
+ StateUpdater updater;
+ State state;
+ String stateId;
+ TridentContext context;
+ Fields inputFields;
+ ProjectionFactory projection;
+ FreshCollector collector;
public PartitionPersistProcessor(String stateId, Fields inputFields, StateUpdater updater) {
- _updater = updater;
- _stateId = stateId;
- _inputFields = inputFields;
+ this.updater = updater;
+ this.stateId = stateId;
+ this.inputFields = inputFields;
}
@Override
@@ -49,26 +49,26 @@
if (parents.size() != 1) {
throw new RuntimeException("Partition persist operation can only have one parent");
}
- _context = tridentContext;
- _state = (State) context.getTaskData(_stateId);
- _projection = new ProjectionFactory(parents.get(0), _inputFields);
- _collector = new FreshCollector(tridentContext);
- _updater.prepare(conf, new TridentOperationContext(context, _projection));
+ this.context = tridentContext;
+ state = (State) context.getTaskData(stateId);
+ projection = new ProjectionFactory(parents.get(0), inputFields);
+ collector = new FreshCollector(tridentContext);
+ updater.prepare(conf, new TridentOperationContext(context, projection));
}
@Override
public void cleanup() {
- _updater.cleanup();
+ updater.cleanup();
}
@Override
public void startBatch(ProcessorContext processorContext) {
- processorContext.state[_context.getStateIndex()] = new ArrayList<TridentTuple>();
+ processorContext.state[context.getStateIndex()] = new ArrayList<TridentTuple>();
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- ((List) processorContext.state[_context.getStateIndex()]).add(_projection.create(tuple));
+ ((List) processorContext.state[context.getStateIndex()]).add(projection.create(tuple));
}
@Override
@@ -78,10 +78,10 @@
@Override
public void finishBatch(ProcessorContext processorContext) {
- _collector.setContext(processorContext);
+ collector.setContext(processorContext);
Object batchId = processorContext.batchId;
// since this processor type is a committer, this occurs in the commit phase
- List<TridentTuple> buffer = (List) processorContext.state[_context.getStateIndex()];
+ List<TridentTuple> buffer = (List) processorContext.state[context.getStateIndex()];
// don't update unless there are tuples
// this helps out with things like global partition persist, where multiple tasks may still
@@ -94,14 +94,14 @@
if (batchId instanceof TransactionAttempt) {
txid = ((TransactionAttempt) batchId).getTransactionId();
}
- _state.beginCommit(txid);
- _updater.updateState(_state, buffer, _collector);
- _state.commit(txid);
+ state.beginCommit(txid);
+ updater.updateState(state, buffer, collector);
+ state.commit(txid);
}
}
@Override
public Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/ProjectedProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/ProjectedProcessor.java
index a026491..1ed9012 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/ProjectedProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/ProjectedProcessor.java
@@ -24,12 +24,12 @@
public class ProjectedProcessor implements TridentProcessor {
- Fields _projectFields;
- ProjectionFactory _factory;
- TridentContext _context;
+ Fields projectFields;
+ ProjectionFactory factory;
+ TridentContext context;
public ProjectedProcessor(Fields projectFields) {
- _projectFields = projectFields;
+ this.projectFields = projectFields;
}
@Override
@@ -37,8 +37,8 @@
if (tridentContext.getParentTupleFactories().size() != 1) {
throw new RuntimeException("Projection processor can only have one parent");
}
- _context = tridentContext;
- _factory = new ProjectionFactory(tridentContext.getParentTupleFactories().get(0), _projectFields);
+ this.context = tridentContext;
+ factory = new ProjectionFactory(tridentContext.getParentTupleFactories().get(0), projectFields);
}
@Override
@@ -51,15 +51,15 @@
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- TridentTuple toEmit = _factory.create(tuple);
- for (TupleReceiver r : _context.getReceivers()) {
- r.execute(processorContext, _context.getOutStreamId(), toEmit);
+ TridentTuple toEmit = factory.create(tuple);
+ for (TupleReceiver r : context.getReceivers()) {
+ r.execute(processorContext, context.getOutStreamId(), toEmit);
}
}
@Override
public void flush() {
- for (TupleReceiver r : _context.getReceivers()) {
+ for (TupleReceiver r : context.getReceivers()) {
r.flush();
}
}
@@ -70,6 +70,6 @@
@Override
public Factory getOutputFactory() {
- return _factory;
+ return factory;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/StateQueryProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/StateQueryProcessor.java
index 75867a9..d9126c1 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/planner/processor/StateQueryProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/planner/processor/StateQueryProcessor.java
@@ -29,18 +29,18 @@
public class StateQueryProcessor implements TridentProcessor {
- QueryFunction _function;
- State _state;
- String _stateId;
- TridentContext _context;
- Fields _inputFields;
- ProjectionFactory _projection;
- AppendCollector _collector;
+ QueryFunction function;
+ State state;
+ String stateId;
+ TridentContext context;
+ Fields inputFields;
+ ProjectionFactory projection;
+ AppendCollector collector;
public StateQueryProcessor(String stateId, Fields inputFields, QueryFunction function) {
- _stateId = stateId;
- _function = function;
- _inputFields = inputFields;
+ this.stateId = stateId;
+ this.function = function;
+ this.inputFields = inputFields;
}
@Override
@@ -49,28 +49,28 @@
if (parents.size() != 1) {
throw new RuntimeException("State query operation can only have one parent");
}
- _context = tridentContext;
- _state = (State) context.getTaskData(_stateId);
- _projection = new ProjectionFactory(parents.get(0), _inputFields);
- _collector = new AppendCollector(tridentContext);
- _function.prepare(conf, new TridentOperationContext(context, _projection));
+ this.context = tridentContext;
+ state = (State) context.getTaskData(stateId);
+ projection = new ProjectionFactory(parents.get(0), inputFields);
+ collector = new AppendCollector(tridentContext);
+ function.prepare(conf, new TridentOperationContext(context, projection));
}
@Override
public void cleanup() {
- _function.cleanup();
+ function.cleanup();
}
@Override
public void startBatch(ProcessorContext processorContext) {
- processorContext.state[_context.getStateIndex()] = new BatchState();
+ processorContext.state[context.getStateIndex()] = new BatchState();
}
@Override
public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) {
- BatchState state = (BatchState) processorContext.state[_context.getStateIndex()];
+ BatchState state = (BatchState) processorContext.state[context.getStateIndex()];
state.tuples.add(tuple);
- state.args.add(_projection.create(tuple));
+ state.args.add(projection.create(tuple));
}
@Override
@@ -80,9 +80,9 @@
@Override
public void finishBatch(ProcessorContext processorContext) {
- BatchState state = (BatchState) processorContext.state[_context.getStateIndex()];
+ BatchState state = (BatchState) processorContext.state[context.getStateIndex()];
if (!state.tuples.isEmpty()) {
- List<Object> results = _function.batchRetrieve(_state, Collections.unmodifiableList(state.args));
+ List<Object> results = function.batchRetrieve(this.state, Collections.unmodifiableList(state.args));
if (results.size() != state.tuples.size()) {
throw new RuntimeException(
"Results size is different than argument size: " + results.size() + " vs " + state.tuples.size());
@@ -90,15 +90,15 @@
for (int i = 0; i < state.tuples.size(); i++) {
TridentTuple tuple = state.tuples.get(i);
Object result = results.get(i);
- _collector.setContext(processorContext, tuple);
- _function.execute(state.args.get(i), result, _collector);
+ collector.setContext(processorContext, tuple);
+ function.execute(state.args.get(i), result, collector);
}
}
}
@Override
public Factory getOutputFactory() {
- return _collector.getOutputFactory();
+ return collector.getOutputFactory();
}
private static class BatchState {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/BatchSpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/spout/BatchSpoutExecutor.java
index 6952cc9..d86269a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/BatchSpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/BatchSpoutExecutor.java
@@ -19,10 +19,10 @@
import org.apache.storm.tuple.Fields;
public class BatchSpoutExecutor implements ITridentSpout<Object> {
- IBatchSpout _spout;
+ IBatchSpout spout;
public BatchSpoutExecutor(IBatchSpout spout) {
- _spout = spout;
+ this.spout = spout;
}
@Override
@@ -32,18 +32,18 @@
@Override
public Emitter<Object> getEmitter(String txStateId, Map<String, Object> conf, TopologyContext context) {
- _spout.open(conf, context);
+ spout.open(conf, context);
return new BatchSpoutEmitter();
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
@Override
public Fields getOutputFields() {
- return _spout.getOutputFields();
+ return spout.getOutputFields();
}
public static class EmptyCoordinator implements BatchCoordinator<Object> {
@@ -70,17 +70,17 @@
@Override
public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) {
- _spout.emitBatch(tx.getTransactionId(), collector);
+ spout.emitBatch(tx.getTransactionId(), collector);
}
@Override
public void success(TransactionAttempt tx) {
- _spout.ack(tx.getTransactionId());
+ spout.ack(tx.getTransactionId());
}
@Override
public void close() {
- _spout.close();
+ spout.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/IBatchID.java b/storm-client/src/jvm/org/apache/storm/trident/spout/IBatchID.java
index da9a4db..f4f5e4f 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/IBatchID.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/IBatchID.java
@@ -13,6 +13,7 @@
package org.apache.storm.trident.spout;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public interface IBatchID {
Object getId();
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java b/storm-client/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java
index b6a0281..6d5c320 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/IOpaquePartitionedTridentSpout.java
@@ -25,15 +25,15 @@
*
* @param <M> The type of metadata object passed to the Emitter when emitting a new batch based on a previous batch. This type must
* be JSON serializable by json-simple.
- * @param <Partitions> The type of metadata object used by the coordinator to describe partitions. This type must be JSON serializable by
+ * @param <PartitionsT> The type of metadata object used by the coordinator to describe partitions. This type must be JSON serializable by
* json-simple.
*/
-public interface IOpaquePartitionedTridentSpout<Partitions, Partition extends ISpoutPartition, M>
+public interface IOpaquePartitionedTridentSpout<PartitionsT, PartitionT extends ISpoutPartition, M>
extends ITridentDataSource {
- Emitter<Partitions, Partition, M> getEmitter(Map<String, Object> conf, TopologyContext context);
+ Emitter<PartitionsT, PartitionT, M> getEmitter(Map<String, Object> conf, TopologyContext context);
- Coordinator<Partitions> getCoordinator(Map<String, Object> conf, TopologyContext context);
+ Coordinator<PartitionsT> getCoordinator(Map<String, Object> conf, TopologyContext context);
Map<String, Object> getComponentConfiguration();
@@ -42,10 +42,10 @@
/**
* Coordinator for batches. Trident will only begin committing once at least one coordinator is ready.
*
- * @param <Partitions> The type of metadata object used by the coordinator to describe partitions. This type must be JSON serializable
+ * @param <PartitionsT> The type of metadata object used by the coordinator to describe partitions. This type must be JSON serializable
* by json-simple.
*/
- interface Coordinator<Partitions> {
+ interface Coordinator<PartitionsT> {
/**
* Indicates whether this coordinator is ready to commit the given transaction. The master batch coordinator will only begin
* committing if at least one coordinator indicates it is ready to commit.
@@ -60,18 +60,19 @@
*
* @return The partitions for the following batches.
*/
- Partitions getPartitionsForBatch();
+ PartitionsT getPartitionsForBatch();
void close();
}
- interface Emitter<Partitions, Partition extends ISpoutPartition, M> {
+ interface Emitter<PartitionsT, PartitionT extends ISpoutPartition, M> {
/**
* Emit a batch of tuples for a partition/transaction.
*
- * Return the metadata describing this batch that will be used as lastPartitionMeta for defining the parameters of the next batch.
+ * <p>Return the metadata describing this batch that will be used as lastPartitionMeta for defining the
+ * parameters of the next batch.
*/
- M emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, M lastPartitionMeta);
+ M emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, PartitionT partition, M lastPartitionMeta);
/**
* This method is called when this task is responsible for a new set of partitions. Should be used to manage things like connections
@@ -79,7 +80,7 @@
*
* @param partitionResponsibilities The partitions assigned to this task
*/
- void refreshPartitions(List<Partition> partitionResponsibilities);
+ void refreshPartitions(List<PartitionT> partitionResponsibilities);
/**
* Sorts the partition info to produce an ordered list of partition.
@@ -87,7 +88,7 @@
* @param allPartitionInfo The partition info for all partitions being processed by all spout tasks
* @return The ordered list of partitions being processed by all the tasks. The ordering must be consistent for all tasks.
*/
- List<Partition> getOrderedPartitions(Partitions allPartitionInfo);
+ List<PartitionT> getOrderedPartitions(PartitionsT allPartitionInfo);
/**
* Get the partitions assigned to this task.
@@ -98,8 +99,8 @@
* {@link #getOrderedPartitions(java.lang.Object)}
* @return The list of partitions that are to be processed by the task with id {@code taskId}
*/
- default List<Partition> getPartitionsForTask(int taskId, int numTasks, List<Partition> allPartitionInfoSorted) {
- final List<Partition> taskPartitions = new ArrayList<>(allPartitionInfoSorted == null ? 0 : allPartitionInfoSorted.size());
+ default List<PartitionT> getPartitionsForTask(int taskId, int numTasks, List<PartitionT> allPartitionInfoSorted) {
+ final List<PartitionT> taskPartitions = new ArrayList<>(allPartitionInfoSorted == null ? 0 : allPartitionInfoSorted.size());
if (allPartitionInfoSorted != null) {
for (int i = taskId; i < allPartitionInfoSorted.size(); i += numTasks) {
taskPartitions.add(allPartitionInfoSorted.get(i));
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java b/storm-client/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java
index 22d7dbf..0a06976 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/IPartitionedTridentSpout.java
@@ -25,54 +25,54 @@
* metadata for each partition to ensure that the same batch is always emitted for the same transaction id. The partition metadata is stored
* in Zookeeper.
*/
-public interface IPartitionedTridentSpout<Partitions, Partition extends ISpoutPartition, T> extends ITridentDataSource {
- Coordinator<Partitions> getCoordinator(Map<String, Object> conf, TopologyContext context);
+public interface IPartitionedTridentSpout<PartitionsT, PartitionT extends ISpoutPartition, T> extends ITridentDataSource {
+ Coordinator<PartitionsT> getCoordinator(Map<String, Object> conf, TopologyContext context);
- Emitter<Partitions, Partition, T> getEmitter(Map<String, Object> conf, TopologyContext context);
+ Emitter<PartitionsT, PartitionT, T> getEmitter(Map<String, Object> conf, TopologyContext context);
Map<String, Object> getComponentConfiguration();
Fields getOutputFields();
- interface Coordinator<Partitions> {
+ interface Coordinator<PartitionsT> {
/**
* Return the partitions currently in the source of data. The idea is is that if a new partition is added and a prior transaction is
* replayed, it doesn't emit tuples for the new partition because it knows what partitions were in that transaction.
*/
- Partitions getPartitionsForBatch();
+ PartitionsT getPartitionsForBatch();
boolean isReady(long txid);
void close();
}
- interface Emitter<Partitions, Partition extends ISpoutPartition, X> {
+ interface Emitter<PartitionsT, PartitionT extends ISpoutPartition, X> {
/**
- * Sorts given partition info to produce an ordered list of partitions
+ * Sorts given partition info to produce an ordered list of partitions.
*
* @param allPartitionInfo The partition info for all partitions being processed by all spout tasks
* @return sorted list of partitions being processed by all the tasks. The ordering must be consistent for all tasks.
*/
- List<Partition> getOrderedPartitions(Partitions allPartitionInfo);
+ List<PartitionT> getOrderedPartitions(PartitionsT allPartitionInfo);
/**
* Emit a batch of tuples for a partition/transaction that's never been emitted before. Return the metadata that can be used to
* reconstruct this partition/batch in the future.
*/
- X emitPartitionBatchNew(TransactionAttempt tx, TridentCollector collector, Partition partition, X lastPartitionMeta);
+ X emitPartitionBatchNew(TransactionAttempt tx, TridentCollector collector, PartitionT partition, X lastPartitionMeta);
/**
* This method is called when this task is responsible for a new set of partitions. Should be used to manage things like connections
* to brokers.
*/
- void refreshPartitions(List<Partition> partitionResponsibilities);
+ void refreshPartitions(List<PartitionT> partitionResponsibilities);
/**
* Emit a batch of tuples for a partition/transaction that has been emitted before, using the metadata created when it was first
* emitted.
*/
- void emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, X partitionMeta);
+ void emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, PartitionT partition, X partitionMeta);
/**
* Get the partitions assigned to the given task.
@@ -83,8 +83,8 @@
* {@link #getOrderedPartitions(java.lang.Object)}
* @return The list of partitions that are to be processed by the task with {@code taskId}
*/
- default List<Partition> getPartitionsForTask(int taskId, int numTasks, List<Partition> allPartitionInfoSorted) {
- List<Partition> taskPartitions = new ArrayList<>(allPartitionInfoSorted == null ? 0 : allPartitionInfoSorted.size());
+ default List<PartitionT> getPartitionsForTask(int taskId, int numTasks, List<PartitionT> allPartitionInfoSorted) {
+ List<PartitionT> taskPartitions = new ArrayList<>(allPartitionInfoSorted == null ? 0 : allPartitionInfoSorted.size());
if (allPartitionInfoSorted != null) {
for (int i = taskId; i < allPartitionInfoSorted.size(); i += numTasks) {
taskPartitions.add(allPartitionInfoSorted.get(i));
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java b/storm-client/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java
index 45c05a3..b9ab8d5 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/ITridentSpout.java
@@ -21,12 +21,13 @@
public interface ITridentSpout<T> extends ITridentDataSource {
/**
- * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches of tuples should be emitted. The
- * Coordinator that you provide in a TransactionalSpout provides metadata for each transaction so that the transactions can be replayed
- * in case of failure.
+ * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches of tuples should be
+ * emitted. The Coordinator that you provide in a TransactionalSpout provides metadata for each transaction so that
+ * the transactions can be replayed in case of failure.
*
- * Two instances are requested, one on the master batch coordinator where isReady() is called, and an instance in the coordinator bolt
- * which is used for all other operations. The two instances do not necessarily share a worker JVM.
+ * <p>Two instances are requested, one on the master batch coordinator where isReady() is called, and an instance in
+ * the coordinator bolt which is used for all other operations. The two instances do not necessarily share a worker
+ * JVM.
*
* @param txStateId stream id
* @param conf Storm config map
@@ -36,11 +37,12 @@
BatchCoordinator<T> getCoordinator(String txStateId, Map<String, Object> conf, TopologyContext context);
/**
- * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for emitting batches of tuples
- * for a transaction and must ensure that the same batch of tuples is always emitted for the same transaction id.
+ * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for emitting
+ * batches of tuples for a transaction and must ensure that the same batch of tuples is always emitted for the same
+ * transaction id.
*
- * All emitter tasks get the same transaction metadata. The topology context parameter contains the instance task id that can be used to
- * distribute the work across the tasks.
+ * <p>All emitter tasks get the same transaction metadata. The topology context parameter contains the instance task
+ * id that can be used to distribute the work across the tasks.
*
* @param txStateId stream id
* @param conf Storm config map
@@ -55,13 +57,13 @@
interface BatchCoordinator<X> {
/**
- * Create metadata for this particular transaction id which has never been emitted before. The metadata should contain whatever is
- * necessary to be able to replay the exact batch for the transaction at a later point.
+ * Create metadata for this particular transaction id which has never been emitted before. The metadata should
+ * contain whatever is necessary to be able to replay the exact batch for the transaction at a later point.
*
- * The metadata is stored in Zookeeper.
+ * <p>The metadata is stored in Zookeeper.
*
- * Storm uses JSON encoding to store the metadata. Only simple types such as numbers, booleans, strings, lists, and maps should be
- * used.
+ * <p>Storm uses JSON encoding to store the metadata. Only simple types such as numbers, booleans, strings,
+ * lists, and maps should be used.
*
* @param txid The id of the transaction.
* @param prevMetadata The metadata of the previous transaction
@@ -78,7 +80,7 @@
void success(long txid);
/**
- * hint to Storm if the spout is ready for the transaction id
+ * hint to Storm if the spout is ready for the transaction id.
*
* @param txid the id of the transaction
* @return true, if the spout is ready for the given transaction id
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
index 61eaad7..5fffe1f 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
@@ -12,7 +12,6 @@
package org.apache.storm.trident.spout;
-
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -20,6 +19,7 @@
import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
+
import org.apache.storm.task.TopologyContext;
import org.apache.storm.trident.operation.TridentCollector;
import org.apache.storm.trident.topology.TransactionAttempt;
@@ -31,12 +31,12 @@
public class OpaquePartitionedTridentSpoutExecutor implements ICommitterTridentSpout<Object> {
- protected final Logger LOG = LoggerFactory.getLogger(OpaquePartitionedTridentSpoutExecutor.class);
+ protected static final Logger LOG = LoggerFactory.getLogger(OpaquePartitionedTridentSpoutExecutor.class);
- IOpaquePartitionedTridentSpout<Object, ISpoutPartition, Object> _spout;
+ IOpaquePartitionedTridentSpout<Object, ISpoutPartition, Object> spout;
public OpaquePartitionedTridentSpoutExecutor(IOpaquePartitionedTridentSpout<Object, ISpoutPartition, Object> spout) {
- _spout = spout;
+ this.spout = spout;
}
@Override
@@ -51,12 +51,12 @@
@Override
public Fields getOutputFields() {
- return _spout.getOutputFields();
+ return spout.getOutputFields();
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
static class EmitterPartitionState {
@@ -70,23 +70,23 @@
}
public class Coordinator implements ITridentSpout.BatchCoordinator<Object> {
- IOpaquePartitionedTridentSpout.Coordinator _coordinator;
+ IOpaquePartitionedTridentSpout.Coordinator coordinator;
public Coordinator(Map<String, Object> conf, TopologyContext context) {
- _coordinator = _spout.getCoordinator(conf, context);
+ coordinator = spout.getCoordinator(conf, context);
}
@Override
public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) {
LOG.debug("Initialize Transaction. [txid = {}], [prevMetadata = {}], [currMetadata = {}]", txid, prevMetadata, currMetadata);
- return _coordinator.getPartitionsForBatch();
+ return coordinator.getPartitionsForBatch();
}
@Override
public void close() {
LOG.debug("Closing");
- _coordinator.close();
+ coordinator.close();
LOG.debug("Closed");
}
@@ -97,27 +97,27 @@
@Override
public boolean isReady(long txid) {
- boolean ready = _coordinator.isReady(txid);
+ boolean ready = coordinator.isReady(txid);
LOG.debug("[isReady = {}], [txid = {}]", ready, txid);
return ready;
}
}
public class Emitter implements ICommitterTridentSpout.Emitter {
- IOpaquePartitionedTridentSpout.Emitter<Object, ISpoutPartition, Object> _emitter;
- TransactionalState _state;
- TreeMap<Long, Map<String, Object>> _cachedMetas = new TreeMap<>();
- Map<String, EmitterPartitionState> _partitionStates = new HashMap<>();
- int _index;
- int _numTasks;
- Object _savedCoordinatorMeta = null;
- boolean _changedMeta = false;
+ IOpaquePartitionedTridentSpout.Emitter<Object, ISpoutPartition, Object> emitter;
+ TransactionalState state;
+ TreeMap<Long, Map<String, Object>> cachedMetas = new TreeMap<>();
+ Map<String, EmitterPartitionState> partitionStates = new HashMap<>();
+ int index;
+ int numTasks;
+ Object savedCoordinatorMeta = null;
+ boolean changedMeta = false;
public Emitter(String txStateId, Map<String, Object> conf, TopologyContext context) {
- _emitter = _spout.getEmitter(conf, context);
- _index = context.getThisTaskIndex();
- _numTasks = context.getComponentTasks(context.getThisComponentId()).size();
- _state = TransactionalState.newUserState(conf, txStateId);
+ emitter = spout.getEmitter(conf, context);
+ index = context.getThisTaskIndex();
+ numTasks = context.getComponentTasks(context.getThisComponentId()).size();
+ state = TransactionalState.newUserState(conf, txStateId);
LOG.debug("Created {}", this);
}
@@ -126,23 +126,23 @@
LOG.debug("Emitting Batch. [transaction = {}], [coordinatorMeta = {}], [collector = {}], [{}]",
tx, coordinatorMeta, collector, this);
- if (_savedCoordinatorMeta == null || !_savedCoordinatorMeta.equals(coordinatorMeta)) {
- _partitionStates.clear();
- final List<ISpoutPartition> sortedPartitions = _emitter.getOrderedPartitions(coordinatorMeta);
- final List<ISpoutPartition> taskPartitions = _emitter.getPartitionsForTask(_index, _numTasks, sortedPartitions);
+ if (savedCoordinatorMeta == null || !savedCoordinatorMeta.equals(coordinatorMeta)) {
+ partitionStates.clear();
+ final List<ISpoutPartition> sortedPartitions = emitter.getOrderedPartitions(coordinatorMeta);
+ final List<ISpoutPartition> taskPartitions = emitter.getPartitionsForTask(index, numTasks, sortedPartitions);
for (ISpoutPartition partition : taskPartitions) {
- _partitionStates.put(partition.getId(),
- new EmitterPartitionState(new RotatingTransactionalState(_state, partition.getId()), partition));
+ partitionStates.put(partition.getId(),
+ new EmitterPartitionState(new RotatingTransactionalState(state, partition.getId()), partition));
}
- _emitter.refreshPartitions(taskPartitions);
+ emitter.refreshPartitions(taskPartitions);
- _savedCoordinatorMeta = coordinatorMeta;
- _changedMeta = true;
+ savedCoordinatorMeta = coordinatorMeta;
+ changedMeta = true;
}
Map<String, Object> metas = new HashMap<>();
- _cachedMetas.put(tx.getTransactionId(), metas);
+ cachedMetas.put(tx.getTransactionId(), metas);
- Entry<Long, Map<String, Object>> entry = _cachedMetas.lowerEntry(tx.getTransactionId());
+ Entry<Long, Map<String, Object>> entry = cachedMetas.lowerEntry(tx.getTransactionId());
Map<String, Object> prevCached;
if (entry != null) {
prevCached = entry.getValue();
@@ -150,7 +150,7 @@
prevCached = new HashMap<>();
}
- for (Entry<String, EmitterPartitionState> e : _partitionStates.entrySet()) {
+ for (Entry<String, EmitterPartitionState> e : partitionStates.entrySet()) {
String id = e.getKey();
EmitterPartitionState s = e.getValue();
s.rotatingState.removeState(tx.getTransactionId());
@@ -158,7 +158,7 @@
if (lastMeta == null) {
lastMeta = s.rotatingState.getLastState();
}
- Object meta = _emitter.emitPartitionBatch(tx, collector, s.partition, lastMeta);
+ Object meta = emitter.emitPartitionBatch(tx, collector, s.partition, lastMeta);
metas.put(id, meta);
}
LOG.debug("Emitted Batch. [transaction = {}], [coordinatorMeta = {}], [collector = {}], [{}]",
@@ -167,7 +167,7 @@
@Override
public void success(TransactionAttempt tx) {
- for (EmitterPartitionState state : _partitionStates.values()) {
+ for (EmitterPartitionState state : partitionStates.values()) {
state.rotatingState.cleanupBefore(tx.getTransactionId());
}
LOG.debug("Success transaction {}. [{}]", tx, this);
@@ -184,24 +184,24 @@
// because only a single commit can be happening at once. this is because in order for
// another attempt of the batch to commit, the batch phase must have succeeded in between.
// hence, all tasks for the prior commit must have finished committing (whether successfully or not)
- if (_changedMeta && _index == 0) {
+ if (changedMeta && index == 0) {
Set<String> validIds = new HashSet<>();
- for (ISpoutPartition p : _emitter.getOrderedPartitions(_savedCoordinatorMeta)) {
+ for (ISpoutPartition p : emitter.getOrderedPartitions(savedCoordinatorMeta)) {
validIds.add(p.getId());
}
- for (String existingPartition : _state.list("")) {
+ for (String existingPartition : state.list("")) {
if (!validIds.contains(existingPartition)) {
- RotatingTransactionalState s = new RotatingTransactionalState(_state, existingPartition);
+ RotatingTransactionalState s = new RotatingTransactionalState(state, existingPartition);
s.removeState(attempt.getTransactionId());
}
}
- _changedMeta = false;
+ changedMeta = false;
}
Long txid = attempt.getTransactionId();
- Map<String, Object> metas = _cachedMetas.remove(txid);
+ Map<String, Object> metas = cachedMetas.remove(txid);
for (Entry<String, Object> entry : metas.entrySet()) {
- _partitionStates.get(entry.getKey()).rotatingState.overrideState(txid, entry.getValue());
+ partitionStates.get(entry.getKey()).rotatingState.overrideState(txid, entry.getValue());
}
LOG.debug("Exiting commit method for transaction {}. [{}]", attempt, this);
}
@@ -209,21 +209,21 @@
@Override
public void close() {
LOG.debug("Closing");
- _emitter.close();
+ emitter.close();
LOG.debug("Closed");
}
@Override
public String toString() {
- return "Emitter{" +
- ", _state=" + _state +
- ", _cachedMetas=" + _cachedMetas +
- ", _partitionStates=" + _partitionStates +
- ", _index=" + _index +
- ", _numTasks=" + _numTasks +
- ", _savedCoordinatorMeta=" + _savedCoordinatorMeta +
- ", _changedMeta=" + _changedMeta +
- '}';
+ return "Emitter{"
+ + ", state=" + state
+ + ", cachedMetas=" + cachedMetas
+ + ", partitionStates=" + partitionStates
+ + ", index=" + index
+ + ", numTasks=" + numTasks
+ + ", savedCoordinatorMeta=" + savedCoordinatorMeta
+ + ", changedMeta=" + changedMeta
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java
index d066d4e..d0639b9 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/PartitionedTridentSpoutExecutor.java
@@ -12,7 +12,6 @@
package org.apache.storm.trident.spout;
-import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -29,14 +28,14 @@
public class PartitionedTridentSpoutExecutor implements ITridentSpout<Object> {
private static final Logger LOG = LoggerFactory.getLogger(PartitionedTridentSpoutExecutor.class);
- IPartitionedTridentSpout<Object, ISpoutPartition, Object> _spout;
+ IPartitionedTridentSpout<Object, ISpoutPartition, Object> spout;
public PartitionedTridentSpoutExecutor(IPartitionedTridentSpout<Object, ISpoutPartition, Object> spout) {
- _spout = spout;
+ this.spout = spout;
}
public IPartitionedTridentSpout<Object, ISpoutPartition, Object> getPartitionedSpout() {
- return _spout;
+ return spout;
}
@Override
@@ -51,12 +50,12 @@
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
@Override
public Fields getOutputFields() {
- return _spout.getOutputFields();
+ return spout.getOutputFields();
}
static class EmitterPartitionState {
@@ -70,10 +69,10 @@
}
class Coordinator implements ITridentSpout.BatchCoordinator<Object> {
- private IPartitionedTridentSpout.Coordinator<Object> _coordinator;
+ private IPartitionedTridentSpout.Coordinator<Object> coordinator;
public Coordinator(Map<String, Object> conf, TopologyContext context) {
- _coordinator = _spout.getCoordinator(conf, context);
+ coordinator = spout.getCoordinator(conf, context);
}
@Override
@@ -83,7 +82,7 @@
if (currMetadata != null) {
return currMetadata;
} else {
- return _coordinator.getPartitionsForBatch();
+ return coordinator.getPartitionsForBatch();
}
}
@@ -91,7 +90,7 @@
@Override
public void close() {
LOG.debug("Closing");
- _coordinator.close();
+ coordinator.close();
LOG.debug("Closed");
}
@@ -102,59 +101,59 @@
@Override
public boolean isReady(long txid) {
- boolean ready = _coordinator.isReady(txid);
+ boolean ready = coordinator.isReady(txid);
LOG.debug("isReady = {} ", ready);
return ready;
}
}
class Emitter implements ITridentSpout.Emitter<Object> {
- Object _savedCoordinatorMeta = null;
- private IPartitionedTridentSpout.Emitter<Object, ISpoutPartition, Object> _emitter;
- private TransactionalState _state;
- private Map<String, EmitterPartitionState> _partitionStates = new HashMap<>();
- private int _index;
- private int _numTasks;
+ Object savedCoordinatorMeta = null;
+ private IPartitionedTridentSpout.Emitter<Object, ISpoutPartition, Object> emitter;
+ private TransactionalState state;
+ private Map<String, EmitterPartitionState> partitionStates = new HashMap<>();
+ private int index;
+ private int numTasks;
public Emitter(String txStateId, Map<String, Object> conf, TopologyContext context) {
- _emitter = _spout.getEmitter(conf, context);
- _state = TransactionalState.newUserState(conf, txStateId);
- _index = context.getThisTaskIndex();
- _numTasks = context.getComponentTasks(context.getThisComponentId()).size();
+ emitter = spout.getEmitter(conf, context);
+ state = TransactionalState.newUserState(conf, txStateId);
+ index = context.getThisTaskIndex();
+ numTasks = context.getComponentTasks(context.getThisComponentId()).size();
}
@Override
public void emitBatch(final TransactionAttempt tx, final Object coordinatorMeta, final TridentCollector collector) {
LOG.debug("Emitting Batch. [transaction = {}], [coordinatorMeta = {}], [collector = {}]", tx, coordinatorMeta, collector);
- if (_savedCoordinatorMeta == null || !_savedCoordinatorMeta.equals(coordinatorMeta)) {
- _partitionStates.clear();
- List<ISpoutPartition> taskPartitions = _emitter.getPartitionsForTask(_index, _numTasks,
- _emitter.getOrderedPartitions(coordinatorMeta));
+ if (savedCoordinatorMeta == null || !savedCoordinatorMeta.equals(coordinatorMeta)) {
+ partitionStates.clear();
+ List<ISpoutPartition> taskPartitions = emitter.getPartitionsForTask(index, numTasks,
+ emitter.getOrderedPartitions(coordinatorMeta));
for (ISpoutPartition partition : taskPartitions) {
- _partitionStates.put(partition.getId(),
- new EmitterPartitionState(new RotatingTransactionalState(_state, partition.getId()), partition));
+ partitionStates.put(partition.getId(),
+ new EmitterPartitionState(new RotatingTransactionalState(state, partition.getId()), partition));
}
- _emitter.refreshPartitions(taskPartitions);
- _savedCoordinatorMeta = coordinatorMeta;
+ emitter.refreshPartitions(taskPartitions);
+ savedCoordinatorMeta = coordinatorMeta;
}
- for (EmitterPartitionState s : _partitionStates.values()) {
+ for (EmitterPartitionState s : partitionStates.values()) {
RotatingTransactionalState state = s.rotatingState;
final ISpoutPartition partition = s.partition;
Object meta = state.getStateOrCreate(tx.getTransactionId(),
- new RotatingTransactionalState.StateInitializer() {
- @Override
- public Object init(long txid, Object lastState) {
- return _emitter.emitPartitionBatchNew(tx, collector, partition, lastState);
- }
- });
+ new RotatingTransactionalState.StateInitializer() {
+ @Override
+ public Object init(long txid, Object lastState) {
+ return emitter.emitPartitionBatchNew(tx, collector, partition, lastState);
+ }
+ });
// it's null if one of:
// a) a later transaction batch was emitted before this, so we should skip this batch
// b) if didn't exist and was created (in which case the StateInitializer was invoked and
// it was emitted
if (meta != null) {
- _emitter.emitPartitionBatch(tx, collector, partition, meta);
+ emitter.emitPartitionBatch(tx, collector, partition, meta);
}
}
LOG.debug("Emitted Batch. [tx = {}], [coordinatorMeta = {}], [collector = {}]", tx, coordinatorMeta, collector);
@@ -163,7 +162,7 @@
@Override
public void success(TransactionAttempt tx) {
LOG.debug("Success transaction " + tx);
- for (EmitterPartitionState state : _partitionStates.values()) {
+ for (EmitterPartitionState state : partitionStates.values()) {
state.rotatingState.cleanupBefore(tx.getTransactionId());
}
}
@@ -171,8 +170,8 @@
@Override
public void close() {
LOG.debug("Closing");
- _state.close();
- _emitter.close();
+ state.close();
+ emitter.close();
LOG.debug("Closed");
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchExecutor.java
index 4d843cf..16b86d6 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchExecutor.java
@@ -29,20 +29,20 @@
public class RichSpoutBatchExecutor implements ITridentSpout<Object> {
public static final String MAX_BATCH_SIZE_CONF = "topology.spout.max.batch.size";
- IRichSpout _spout;
+ IRichSpout spout;
public RichSpoutBatchExecutor(IRichSpout spout) {
- _spout = spout;
+ this.spout = spout;
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
@Override
public Fields getOutputFields() {
- return TridentUtils.getSingleOutputStreamFields(_spout);
+ return TridentUtils.getSingleOutputStreamFields(spout);
}
@@ -81,16 +81,16 @@
public List<Object> ids;
public int numEmitted;
public long pendingCount;
- TridentCollector _collector;
+ TridentCollector collector;
public void reset(TridentCollector c) {
- _collector = c;
+ collector = c;
ids = new ArrayList<>();
}
@Override
public void reportError(Throwable t) {
- _collector.reportError(t);
+ collector.reportError(t);
}
@Override
@@ -99,7 +99,7 @@
ids.add(id);
}
numEmitted++;
- _collector.emit(values);
+ collector.emit(values);
return null;
}
@@ -110,7 +110,7 @@
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
@@ -120,24 +120,24 @@
}
class RichSpoutEmitter implements ITridentSpout.Emitter<Object> {
- int _maxBatchSize;
+ int maxBatchSize;
boolean prepared = false;
- CaptureCollector _collector;
+ CaptureCollector collector;
RotatingMap<Long, List<Object>> idsMap;
- Map _conf;
- TopologyContext _context;
+ Map conf;
+ TopologyContext context;
long lastRotate = System.currentTimeMillis();
long rotateTime;
public RichSpoutEmitter(Map<String, Object> conf, TopologyContext context) {
- _conf = conf;
- _context = context;
+ this.conf = conf;
+ this.context = context;
Number batchSize = (Number) conf.get(MAX_BATCH_SIZE_CONF);
if (batchSize == null) {
batchSize = 1000;
}
- _maxBatchSize = batchSize.intValue();
- _collector = new CaptureCollector();
+ maxBatchSize = batchSize.intValue();
+ collector = new CaptureCollector();
idsMap = new RotatingMap<>(3);
rotateTime = 1000L * ((Number) conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)).intValue();
}
@@ -160,19 +160,19 @@
fail(txid);
}
- _collector.reset(collector);
+ this.collector.reset(collector);
if (!prepared) {
- _spout.open(_conf, _context, new SpoutOutputCollector(_collector));
+ spout.open(conf, context, new SpoutOutputCollector(this.collector));
prepared = true;
}
- for (int i = 0; i < _maxBatchSize; i++) {
- _spout.nextTuple();
- if (_collector.numEmitted < i) {
+ for (int i = 0; i < maxBatchSize; i++) {
+ spout.nextTuple();
+ if (this.collector.numEmitted < i) {
break;
}
}
- idsMap.put(txid, _collector.ids);
- _collector.pendingCount = idsMap.size();
+ idsMap.put(txid, this.collector.ids);
+ this.collector.pendingCount = idsMap.size();
}
@@ -185,7 +185,7 @@
List<Object> ids = (List<Object>) idsMap.remove(batchId);
if (ids != null) {
for (Object id : ids) {
- _spout.ack(id);
+ spout.ack(id);
}
}
}
@@ -194,14 +194,14 @@
List<Object> ids = (List<Object>) idsMap.remove(batchId);
if (ids != null) {
for (Object id : ids) {
- _spout.fail(id);
+ spout.fail(id);
}
}
}
@Override
public void close() {
- _spout.close();
+ spout.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchId.java b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchId.java
index 0cf83dd..22b4441 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchId.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchId.java
@@ -13,10 +13,10 @@
package org.apache.storm.trident.spout;
public class RichSpoutBatchId implements IBatchID {
- long _id;
+ long id;
public RichSpoutBatchId(long id) {
- _id = id;
+ this.id = id;
}
@Override
@@ -32,7 +32,7 @@
@Override
public int hashCode() {
- return ((Long) _id).hashCode();
+ return ((Long) id).hashCode();
}
@Override
@@ -41,6 +41,6 @@
return false;
}
RichSpoutBatchId other = (RichSpoutBatchId) o;
- return _id == other._id;
+ return id == other.id;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchIdSerializer.java b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchIdSerializer.java
index 07dae6f..511a0eb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchIdSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchIdSerializer.java
@@ -22,7 +22,7 @@
@Override
public void write(Kryo kryo, Output output, RichSpoutBatchId id) {
- output.writeLong(id._id);
+ output.writeLong(id.id);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchTriggerer.java b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchTriggerer.java
index e5a2d3e..aa0bddf 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchTriggerer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/RichSpoutBatchTriggerer.java
@@ -36,86 +36,86 @@
public class RichSpoutBatchTriggerer implements IRichSpout {
- String _stream;
- IRichSpout _delegate;
- List<Integer> _outputTasks;
- Random _rand;
- String _coordStream;
- Map<Long, Long> _msgIdToBatchId = new HashMap<>();
- Map<Long, FinishCondition> _finishConditions = new HashMap<>();
+ String stream;
+ IRichSpout delegate;
+ List<Integer> outputTasks;
+ Random rand;
+ String coordStream;
+ Map<Long, Long> msgIdToBatchId = new HashMap<>();
+ Map<Long, FinishCondition> finishConditions = new HashMap<>();
public RichSpoutBatchTriggerer(IRichSpout delegate, String streamName, String batchGroup) {
- _delegate = delegate;
- _stream = streamName;
- _coordStream = TridentBoltExecutor.COORD_STREAM(batchGroup);
+ this.delegate = delegate;
+ stream = streamName;
+ coordStream = TridentBoltExecutor.coordStream(batchGroup);
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _delegate.open(conf, context, new SpoutOutputCollector(new StreamOverrideCollector(collector)));
- _outputTasks = new ArrayList<>();
+ delegate.open(conf, context, new SpoutOutputCollector(new StreamOverrideCollector(collector)));
+ outputTasks = new ArrayList<>();
for (String component : Utils.get(context.getThisTargets(),
- _coordStream,
+ coordStream,
new HashMap<String, Grouping>()).keySet()) {
- _outputTasks.addAll(context.getComponentTasks(component));
+ outputTasks.addAll(context.getComponentTasks(component));
}
- _rand = new Random(Utils.secureRandomLong());
+ rand = new Random(Utils.secureRandomLong());
}
@Override
public void close() {
- _delegate.close();
+ delegate.close();
}
@Override
public void activate() {
- _delegate.activate();
+ delegate.activate();
}
@Override
public void deactivate() {
- _delegate.deactivate();
+ delegate.deactivate();
}
@Override
public void nextTuple() {
- _delegate.nextTuple();
+ delegate.nextTuple();
}
@Override
public void ack(Object msgId) {
- Long batchId = _msgIdToBatchId.remove((Long) msgId);
- FinishCondition cond = _finishConditions.get(batchId);
+ Long batchId = msgIdToBatchId.remove((Long) msgId);
+ FinishCondition cond = finishConditions.get(batchId);
if (cond != null) {
cond.vals.remove((Long) msgId);
if (cond.vals.isEmpty()) {
- _finishConditions.remove(batchId);
- _delegate.ack(cond.msgId);
+ finishConditions.remove(batchId);
+ delegate.ack(cond.msgId);
}
}
}
@Override
public void fail(Object msgId) {
- Long batchId = _msgIdToBatchId.remove((Long) msgId);
- FinishCondition cond = _finishConditions.remove(batchId);
+ Long batchId = msgIdToBatchId.remove((Long) msgId);
+ FinishCondition cond = finishConditions.remove(batchId);
if (cond != null) {
- _delegate.fail(cond.msgId);
+ delegate.fail(cond.msgId);
}
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- Fields outFields = TridentUtils.getSingleOutputStreamFields(_delegate);
+ Fields outFields = TridentUtils.getSingleOutputStreamFields(delegate);
outFields = TridentUtils.fieldsConcat(new Fields("$id$"), outFields);
- declarer.declareStream(_stream, outFields);
+ declarer.declareStream(stream, outFields);
// try to find a way to merge this code with what's already done in TridentBoltExecutor
- declarer.declareStream(_coordStream, true, new Fields("id", "count"));
+ declarer.declareStream(coordStream, true, new Fields("id", "count"));
}
@Override
public Map<String, Object> getComponentConfiguration() {
- Map<String, Object> conf = _delegate.getComponentConfiguration();
+ Map<String, Object> conf = delegate.getComponentConfiguration();
if (conf == null) {
conf = new HashMap<>();
} else {
@@ -132,31 +132,31 @@
class StreamOverrideCollector implements ISpoutOutputCollector {
- SpoutOutputCollector _collector;
+ SpoutOutputCollector collector;
public StreamOverrideCollector(SpoutOutputCollector collector) {
- _collector = collector;
+ this.collector = collector;
}
@Override
public List<Integer> emit(String ignore, List<Object> values, Object msgId) {
- long batchIdVal = _rand.nextLong();
+ long batchIdVal = rand.nextLong();
Object batchId = new RichSpoutBatchId(batchIdVal);
FinishCondition finish = new FinishCondition();
finish.msgId = msgId;
- List<Integer> tasks = _collector.emit(_stream, new ConsList(batchId, values));
+ List<Integer> tasks = collector.emit(stream, new ConsList(batchId, values));
Set<Integer> outTasksSet = new HashSet<>(tasks);
- for (Integer t : _outputTasks) {
+ for (Integer t : outputTasks) {
int count = 0;
if (outTasksSet.contains(t)) {
count = 1;
}
- long r = _rand.nextLong();
- _collector.emitDirect(t, _coordStream, new Values(batchId, count), r);
+ long r = rand.nextLong();
+ collector.emitDirect(t, coordStream, new Values(batchId, count), r);
finish.vals.add(r);
- _msgIdToBatchId.put(r, batchIdVal);
+ msgIdToBatchId.put(r, batchIdVal);
}
- _finishConditions.put(batchIdVal, finish);
+ finishConditions.put(batchIdVal, finish);
return tasks;
}
@@ -167,17 +167,17 @@
@Override
public void flush() {
- _collector.flush();
+ collector.flush();
}
@Override
public void reportError(Throwable t) {
- _collector.reportError(t);
+ collector.reportError(t);
}
@Override
public long getPendingCount() {
- return _collector.getPendingCount();
+ return collector.getPendingCount();
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutCoordinator.java b/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutCoordinator.java
index 743d017..7230726 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutCoordinator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutCoordinator.java
@@ -33,23 +33,23 @@
public static final Logger LOG = LoggerFactory.getLogger(TridentSpoutCoordinator.class);
private static final String META_DIR = "meta";
- ITridentSpout<Object> _spout;
- ITridentSpout.BatchCoordinator<Object> _coord;
- RotatingTransactionalState _state;
- TransactionalState _underlyingState;
- String _id;
+ ITridentSpout<Object> spout;
+ ITridentSpout.BatchCoordinator<Object> coord;
+ RotatingTransactionalState state;
+ TransactionalState underlyingState;
+ String id;
public TridentSpoutCoordinator(String id, ITridentSpout<Object> spout) {
- _spout = spout;
- _id = id;
+ this.spout = spout;
+ this.id = id;
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context) {
- _coord = _spout.getCoordinator(_id, conf, context);
- _underlyingState = TransactionalState.newCoordinatorState(conf, _id);
- _state = new RotatingTransactionalState(_underlyingState, META_DIR);
+ coord = spout.getCoordinator(id, conf, context);
+ underlyingState = TransactionalState.newCoordinatorState(conf, id);
+ state = new RotatingTransactionalState(underlyingState, META_DIR);
}
@Override
@@ -57,13 +57,13 @@
TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0);
if (tuple.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) {
- _state.cleanupBefore(attempt.getTransactionId());
- _coord.success(attempt.getTransactionId());
+ state.cleanupBefore(attempt.getTransactionId());
+ coord.success(attempt.getTransactionId());
} else {
long txid = attempt.getTransactionId();
- Object prevMeta = _state.getPreviousState(txid);
- Object meta = _coord.initializeTransaction(txid, prevMeta, _state.getState(txid));
- _state.overrideState(txid, meta);
+ Object prevMeta = state.getPreviousState(txid);
+ Object meta = coord.initializeTransaction(txid, prevMeta, state.getState(txid));
+ state.overrideState(txid, meta);
collector.emit(MasterBatchCoordinator.BATCH_STREAM_ID, new Values(attempt, meta));
}
@@ -71,8 +71,8 @@
@Override
public void cleanup() {
- _coord.close();
- _underlyingState.close();
+ coord.close();
+ underlyingState.close();
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutExecutor.java
index 2f57e0a..42223a7 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/spout/TridentSpoutExecutor.java
@@ -36,24 +36,24 @@
public static final Logger LOG = LoggerFactory.getLogger(TridentSpoutExecutor.class);
- AddIdCollector _collector;
- ITridentSpout<Object> _spout;
- ITridentSpout.Emitter<Object> _emitter;
- String _streamName;
- String _txStateId;
+ AddIdCollector collector;
+ ITridentSpout<Object> spout;
+ ITridentSpout.Emitter<Object> emitter;
+ String streamName;
+ String txStateId;
- TreeMap<Long, TransactionAttempt> _activeBatches = new TreeMap<>();
+ TreeMap<Long, TransactionAttempt> activeBatches = new TreeMap<>();
public TridentSpoutExecutor(String txStateId, String streamName, ITridentSpout<Object> spout) {
- _txStateId = txStateId;
- _spout = spout;
- _streamName = streamName;
+ this.txStateId = txStateId;
+ this.spout = spout;
+ this.streamName = streamName;
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context, BatchOutputCollector collector) {
- _emitter = _spout.getEmitter(_txStateId, conf, context);
- _collector = new AddIdCollector(_streamName, collector);
+ emitter = spout.getEmitter(txStateId, conf, context);
+ this.collector = new AddIdCollector(streamName, collector);
}
@Override
@@ -61,39 +61,39 @@
// there won't be a BatchInfo for the success stream
TransactionAttempt attempt = (TransactionAttempt) input.getValue(0);
if (input.getSourceStreamId().equals(MasterBatchCoordinator.COMMIT_STREAM_ID)) {
- if (attempt.equals(_activeBatches.get(attempt.getTransactionId()))) {
- ((ICommitterTridentSpout.Emitter) _emitter).commit(attempt);
- _activeBatches.remove(attempt.getTransactionId());
+ if (attempt.equals(activeBatches.get(attempt.getTransactionId()))) {
+ ((ICommitterTridentSpout.Emitter) emitter).commit(attempt);
+ activeBatches.remove(attempt.getTransactionId());
} else {
throw new FailedException("Received commit for different transaction attempt");
}
} else if (input.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) {
// valid to delete before what's been committed since
// those batches will never be accessed again
- _activeBatches.headMap(attempt.getTransactionId()).clear();
- _emitter.success(attempt);
+ activeBatches.headMap(attempt.getTransactionId()).clear();
+ emitter.success(attempt);
} else {
- _collector.setBatch(info.batchId);
- _emitter.emitBatch(attempt, input.getValue(1), _collector);
- _activeBatches.put(attempt.getTransactionId(), attempt);
+ collector.setBatch(info.batchId);
+ emitter.emitBatch(attempt, input.getValue(1), collector);
+ activeBatches.put(attempt.getTransactionId(), attempt);
}
}
@Override
public void cleanup() {
- _emitter.close();
+ emitter.close();
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- List<String> fields = new ArrayList<>(_spout.getOutputFields().toList());
+ List<String> fields = new ArrayList<>(spout.getOutputFields().toList());
fields.add(0, ID_FIELD);
- declarer.declareStream(_streamName, new Fields(fields));
+ declarer.declareStream(streamName, new Fields(fields));
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
@Override
@@ -106,33 +106,33 @@
}
private static class AddIdCollector implements TridentCollector {
- BatchOutputCollector _delegate;
- Object _id;
- String _stream;
+ BatchOutputCollector delegate;
+ Object id;
+ String stream;
public AddIdCollector(String stream, BatchOutputCollector c) {
- _delegate = c;
- _stream = stream;
+ delegate = c;
+ this.stream = stream;
}
public void setBatch(Object id) {
- _id = id;
+ this.id = id;
}
@Override
public void emit(List<Object> values) {
- _delegate.emit(_stream, new ConsList(_id, values));
+ delegate.emit(stream, new ConsList(id, values));
}
@Override
public void flush() {
- _delegate.flush();
+ delegate.flush();
}
@Override
public void reportError(Throwable t) {
- _delegate.reportError(t);
+ delegate.reportError(t);
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/JSONNonTransactionalSerializer.java b/storm-client/src/jvm/org/apache/storm/trident/state/JSONNonTransactionalSerializer.java
index 6bc9db4..a56d393 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/JSONNonTransactionalSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/JSONNonTransactionalSerializer.java
@@ -16,6 +16,7 @@
import org.apache.storm.shade.org.json.simple.JSONValue;
import org.apache.storm.shade.org.json.simple.parser.ParseException;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class JSONNonTransactionalSerializer implements Serializer {
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/JSONOpaqueSerializer.java b/storm-client/src/jvm/org/apache/storm/trident/state/JSONOpaqueSerializer.java
index 7cc6467..ec94d3a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/JSONOpaqueSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/JSONOpaqueSerializer.java
@@ -18,6 +18,7 @@
import org.apache.storm.shade.org.json.simple.JSONValue;
import org.apache.storm.shade.org.json.simple.parser.ParseException;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class JSONOpaqueSerializer implements Serializer<OpaqueValue> {
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/JSONTransactionalSerializer.java b/storm-client/src/jvm/org/apache/storm/trident/state/JSONTransactionalSerializer.java
index a9067ec..07d6d3a 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/JSONTransactionalSerializer.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/JSONTransactionalSerializer.java
@@ -18,6 +18,7 @@
import org.apache.storm.shade.org.json.simple.JSONValue;
import org.apache.storm.shade.org.json.simple.parser.ParseException;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class JSONTransactionalSerializer implements Serializer<TransactionalValue> {
@Override
public byte[] serialize(TransactionalValue obj) {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/State.java b/storm-client/src/jvm/org/apache/storm/trident/state/State.java
index ff475c7..19cf17d 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/State.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/State.java
@@ -15,14 +15,14 @@
/**
* There's 3 different kinds of state:
*
- * 1. non-transactional: ignores commits, updates are permanent. no rollback. a cassandra incrementing state would be like this 2.
+ * <p>1. non-transactional: ignores commits, updates are permanent. no rollback. a cassandra incrementing state would be like this 2.
* repeat-transactional: idempotent as long as all batches for a txid are identical 3. opaque-transactional: the most general kind of state.
* updates are always done based on the previous version of the value if the current commit = latest stored commit Idempotent even if the
* batch for a txid can change.
*
- * repeat transactional is idempotent for transactional spouts opaque transactional is idempotent for opaque or transactional spouts
+ * <p>repeat transactional is idempotent for transactional spouts opaque transactional is idempotent for opaque or transactional spouts
*
- * Trident should log warnings when state is idempotent but updates will not be idempotent because of spout
+ * <p>Trident should log warnings when state is idempotent but updates will not be idempotent because of spout
*/
// retrieving is encapsulated in Retrieval interface
public interface State {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedBatchReadsMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedBatchReadsMap.java
index cdce9b8..bed36df 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedBatchReadsMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedBatchReadsMap.java
@@ -19,25 +19,25 @@
public class CachedBatchReadsMap<T> {
- public IBackingMap<T> _delegate;
- Map<List<Object>, T> _cached = new HashMap<List<Object>, T>();
+ public IBackingMap<T> delegate;
+ Map<List<Object>, T> cached = new HashMap<List<Object>, T>();
public CachedBatchReadsMap(IBackingMap<T> delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
public void reset() {
- _cached.clear();
+ cached.clear();
}
public List<RetVal<T>> multiGet(List<List<Object>> keys) {
// TODO: can optimize further by only querying backing map for keys not in the cache
- List<T> vals = _delegate.multiGet(keys);
+ List<T> vals = delegate.multiGet(keys);
List<RetVal<T>> ret = new ArrayList(vals.size());
for (int i = 0; i < keys.size(); i++) {
List<Object> key = keys.get(i);
- if (_cached.containsKey(key)) {
- ret.add(new RetVal(_cached.get(key), true));
+ if (cached.containsKey(key)) {
+ ret.add(new RetVal(cached.get(key), true));
} else {
ret.add(new RetVal(vals.get(i), false));
}
@@ -46,7 +46,7 @@
}
public void multiPut(List<List<Object>> keys, List<T> vals) {
- _delegate.multiPut(keys, vals);
+ delegate.multiPut(keys, vals);
cache(keys, vals);
}
@@ -54,7 +54,7 @@
for (int i = 0; i < keys.size(); i++) {
List<Object> key = keys.get(i);
T val = vals.get(i);
- _cached.put(key, val);
+ cached.put(key, val);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedMap.java
index d6d9956..a0134a4 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/CachedMap.java
@@ -19,17 +19,15 @@
import org.apache.storm.trident.util.LRUMap;
/**
- * Useful to layer over a map that communicates with a database. you generally layer opaque map over this over your database store
- *
- * @param <T>
+ * Useful to layer over a map that communicates with a database. you generally layer opaque map over this over your database store.
*/
public class CachedMap<T> implements IBackingMap<T> {
- LRUMap<List<Object>, T> _cache;
- IBackingMap<T> _delegate;
+ LRUMap<List<Object>, T> cache;
+ IBackingMap<T> delegate;
public CachedMap(IBackingMap<T> delegate, int cacheSize) {
- _cache = new LRUMap<List<Object>, T>(cacheSize);
- _delegate = delegate;
+ cache = new LRUMap<List<Object>, T>(cacheSize);
+ this.delegate = delegate;
}
@Override
@@ -37,18 +35,18 @@
Map<List<Object>, T> results = new HashMap<List<Object>, T>();
List<List<Object>> toGet = new ArrayList<List<Object>>();
for (List<Object> key : keys) {
- if (_cache.containsKey(key)) {
- results.put(key, _cache.get(key));
+ if (cache.containsKey(key)) {
+ results.put(key, cache.get(key));
} else {
toGet.add(key);
}
}
- List<T> fetchedVals = _delegate.multiGet(toGet);
+ List<T> fetchedVals = delegate.multiGet(toGet);
for (int i = 0; i < toGet.size(); i++) {
List<Object> key = toGet.get(i);
T val = fetchedVals.get(i);
- _cache.put(key, val);
+ cache.put(key, val);
results.put(key, val);
}
@@ -62,12 +60,12 @@
@Override
public void multiPut(List<List<Object>> keys, List<T> values) {
cache(keys, values);
- _delegate.multiPut(keys, values);
+ delegate.multiPut(keys, values);
}
private void cache(List<List<Object>> keys, List<T> values) {
for (int i = 0; i < keys.size(); i++) {
- _cache.put(keys.get(i), values.get(i));
+ cache.put(keys.get(i), values.get(i));
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/MapCombinerAggStateUpdater.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/MapCombinerAggStateUpdater.java
index 5046c7e..330ce2f 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/MapCombinerAggStateUpdater.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/MapCombinerAggStateUpdater.java
@@ -31,22 +31,22 @@
//ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
private static final long serialVersionUID = -3960578785572592092L;
- CombinerAggregator _agg;
- Fields _groupFields;
- Fields _inputFields;
- transient ProjectionFactory _groupFactory;
- transient ProjectionFactory _inputFactory;
- ComboList.Factory _factory;
+ CombinerAggregator agg;
+ Fields groupFields;
+ Fields inputFields;
+ transient ProjectionFactory groupFactory;
+ transient ProjectionFactory inputFactory;
+ ComboList.Factory factory;
public MapCombinerAggStateUpdater(CombinerAggregator agg, Fields groupFields, Fields inputFields) {
- _agg = agg;
- _groupFields = groupFields;
- _inputFields = inputFields;
+ this.agg = agg;
+ this.groupFields = groupFields;
+ this.inputFields = inputFields;
if (inputFields.size() != 1) {
throw new IllegalArgumentException(
"Combiner aggs only take a single field as input. Got this instead: " + inputFields.toString());
}
- _factory = new ComboList.Factory(groupFields.size(), inputFields.size());
+ factory = new ComboList.Factory(groupFields.size(), inputFields.size());
}
@Override
@@ -55,22 +55,22 @@
List<ValueUpdater> updaters = new ArrayList<ValueUpdater>(tuples.size());
for (TridentTuple t : tuples) {
- groups.add(_groupFactory.create(t));
- updaters.add(new CombinerValueUpdater(_agg, _inputFactory.create(t).getValue(0)));
+ groups.add(groupFactory.create(t));
+ updaters.add(new CombinerValueUpdater(agg, inputFactory.create(t).getValue(0)));
}
List<Object> newVals = map.multiUpdate(groups, updaters);
for (int i = 0; i < tuples.size(); i++) {
List<Object> key = groups.get(i);
Object result = newVals.get(i);
- collector.emit(_factory.create(new List[]{ key, new Values(result) }));
+ collector.emit(factory.create(new List[]{ key, new Values(result) }));
}
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _groupFactory = context.makeProjectionFactory(_groupFields);
- _inputFactory = context.makeProjectionFactory(_inputFields);
+ groupFactory = context.makeProjectionFactory(groupFields);
+ inputFactory = context.makeProjectionFactory(inputFields);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/MapReducerAggStateUpdater.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/MapReducerAggStateUpdater.java
index 1844acd..1b06c5c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/MapReducerAggStateUpdater.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/MapReducerAggStateUpdater.java
@@ -32,18 +32,18 @@
//ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
private static final long serialVersionUID = 8667174018978959987L;
- ReducerAggregator _agg;
- Fields _groupFields;
- Fields _inputFields;
- transient ProjectionFactory _groupFactory;
- transient ProjectionFactory _inputFactory;
- ComboList.Factory _factory;
+ ReducerAggregator agg;
+ Fields groupFields;
+ Fields inputFields;
+ transient ProjectionFactory groupFactory;
+ transient ProjectionFactory inputFactory;
+ ComboList.Factory factory;
public MapReducerAggStateUpdater(ReducerAggregator agg, Fields groupFields, Fields inputFields) {
- _agg = agg;
- _groupFields = groupFields;
- _inputFields = inputFields;
- _factory = new ComboList.Factory(groupFields.size(), 1);
+ this.agg = agg;
+ this.groupFields = groupFields;
+ this.inputFields = inputFields;
+ factory = new ComboList.Factory(groupFields.size(), 1);
}
@Override
@@ -51,32 +51,32 @@
Map<List<Object>, List<TridentTuple>> grouped = new HashMap<>();
for (TridentTuple t : tuples) {
- List<Object> group = _groupFactory.create(t);
+ List<Object> group = groupFactory.create(t);
List<TridentTuple> groupTuples = grouped.get(group);
if (groupTuples == null) {
groupTuples = new ArrayList<>();
grouped.put(group, groupTuples);
}
- groupTuples.add(_inputFactory.create(t));
+ groupTuples.add(inputFactory.create(t));
}
List<List<Object>> uniqueGroups = new ArrayList<>(grouped.keySet());
List<ValueUpdater> updaters = new ArrayList<>(uniqueGroups.size());
for (List<Object> group : uniqueGroups) {
- updaters.add(new ReducerValueUpdater(_agg, grouped.get(group)));
+ updaters.add(new ReducerValueUpdater(agg, grouped.get(group)));
}
List<Object> results = map.multiUpdate(uniqueGroups, updaters);
for (int i = 0; i < uniqueGroups.size(); i++) {
List<Object> group = uniqueGroups.get(i);
Object result = results.get(i);
- collector.emit(_factory.create(new List[]{ group, new Values(result) }));
+ collector.emit(factory.create(new List[]{ group, new Values(result) }));
}
}
@Override
public void prepare(Map<String, Object> conf, TridentOperationContext context) {
- _groupFactory = context.makeProjectionFactory(_groupFields);
- _inputFactory = context.makeProjectionFactory(_inputFields);
+ groupFactory = context.makeProjectionFactory(groupFields);
+ inputFactory = context.makeProjectionFactory(inputFields);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/MicroBatchIBackingMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/MicroBatchIBackingMap.java
index 0424e0f..73270a8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/MicroBatchIBackingMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/MicroBatchIBackingMap.java
@@ -18,13 +18,13 @@
import java.util.List;
public class MicroBatchIBackingMap<T> implements IBackingMap<T> {
- IBackingMap<T> _delegate;
- Options _options;
+ IBackingMap<T> delegate;
+ Options options;
public MicroBatchIBackingMap(final Options options, final IBackingMap<T> delegate) {
- _options = options;
- _delegate = delegate;
+ this.options = options;
+ this.delegate = delegate;
assert options.maxMultiPutBatchSize >= 0;
assert options.maxMultiGetBatchSize >= 0;
}
@@ -32,10 +32,10 @@
@Override
public void multiPut(final List<List<Object>> keys, final List<T> values) {
int thisBatchSize;
- if (_options.maxMultiPutBatchSize == 0) {
+ if (options.maxMultiPutBatchSize == 0) {
thisBatchSize = keys.size();
} else {
- thisBatchSize = _options.maxMultiPutBatchSize;
+ thisBatchSize = options.maxMultiPutBatchSize;
}
LinkedList<List<Object>> keysTodo = new LinkedList<List<Object>>(keys);
@@ -49,17 +49,17 @@
valuesBatch.add(valuesTodo.removeFirst());
}
- _delegate.multiPut(keysBatch, valuesBatch);
+ delegate.multiPut(keysBatch, valuesBatch);
}
}
@Override
public List<T> multiGet(final List<List<Object>> keys) {
int thisBatchSize;
- if (_options.maxMultiGetBatchSize == 0) {
+ if (options.maxMultiGetBatchSize == 0) {
thisBatchSize = keys.size();
} else {
- thisBatchSize = _options.maxMultiGetBatchSize;
+ thisBatchSize = options.maxMultiGetBatchSize;
}
LinkedList<List<Object>> keysTodo = new LinkedList<List<Object>>(keys);
@@ -72,7 +72,7 @@
keysBatch.add(keysTodo.removeFirst());
}
- List<T> retSubset = _delegate.multiGet(keysBatch);
+ List<T> retSubset = delegate.multiGet(keysBatch);
ret.addAll(retSubset);
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/NonTransactionalMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/NonTransactionalMap.java
index 835395a..a574054 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/NonTransactionalMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/NonTransactionalMap.java
@@ -18,10 +18,10 @@
public class NonTransactionalMap<T> implements MapState<T> {
- IBackingMap<T> _backing;
+ IBackingMap<T> backing;
protected NonTransactionalMap(IBackingMap<T> backing) {
- _backing = backing;
+ this.backing = backing;
}
public static <T> MapState<T> build(IBackingMap<T> backing) {
@@ -30,25 +30,25 @@
@Override
public List<T> multiGet(List<List<Object>> keys) {
- return _backing.multiGet(keys);
+ return backing.multiGet(keys);
}
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- List<T> curr = _backing.multiGet(keys);
+ List<T> curr = backing.multiGet(keys);
List<T> ret = new ArrayList<T>(curr.size());
for (int i = 0; i < curr.size(); i++) {
T currVal = curr.get(i);
ValueUpdater<T> updater = updaters.get(i);
ret.add(updater.update(currVal));
}
- _backing.multiPut(keys, ret);
+ backing.multiPut(keys, ret);
return ret;
}
@Override
public void multiPut(List<List<Object>> keys, List<T> vals) {
- _backing.multiPut(keys, vals);
+ backing.multiPut(keys, vals);
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/OpaqueMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/OpaqueMap.java
index 7f86476..aeda9f4 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/OpaqueMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/OpaqueMap.java
@@ -19,11 +19,11 @@
public class OpaqueMap<T> implements MapState<T> {
- CachedBatchReadsMap<OpaqueValue> _backing;
- Long _currTx;
+ CachedBatchReadsMap<OpaqueValue> backing;
+ Long currTx;
protected OpaqueMap(IBackingMap<OpaqueValue> backing) {
- _backing = new CachedBatchReadsMap(backing);
+ this.backing = new CachedBatchReadsMap(backing);
}
public static <T> MapState<T> build(IBackingMap<OpaqueValue> backing) {
@@ -32,7 +32,7 @@
@Override
public List<T> multiGet(List<List<Object>> keys) {
- List<CachedBatchReadsMap.RetVal<OpaqueValue>> curr = _backing.multiGet(keys);
+ List<CachedBatchReadsMap.RetVal<OpaqueValue>> curr = backing.multiGet(keys);
List<T> ret = new ArrayList<T>(curr.size());
for (CachedBatchReadsMap.RetVal<OpaqueValue> retval : curr) {
OpaqueValue val = retval.val;
@@ -40,7 +40,7 @@
if (retval.cached) {
ret.add((T) val.getCurr());
} else {
- ret.add((T) val.get(_currTx));
+ ret.add((T) val.get(currTx));
}
} else {
ret.add(null);
@@ -51,7 +51,7 @@
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- List<CachedBatchReadsMap.RetVal<OpaqueValue>> curr = _backing.multiGet(keys);
+ List<CachedBatchReadsMap.RetVal<OpaqueValue>> curr = backing.multiGet(keys);
List<OpaqueValue> newVals = new ArrayList<OpaqueValue>(curr.size());
List<T> ret = new ArrayList<T>();
for (int i = 0; i < curr.size(); i++) {
@@ -65,20 +65,20 @@
if (retval.cached) {
prev = val.getCurr();
} else {
- prev = val.get(_currTx);
+ prev = val.get(currTx);
}
}
T newVal = updater.update(prev);
ret.add(newVal);
OpaqueValue<T> newOpaqueVal;
if (val == null) {
- newOpaqueVal = new OpaqueValue<T>(_currTx, newVal);
+ newOpaqueVal = new OpaqueValue<T>(currTx, newVal);
} else {
- newOpaqueVal = val.update(_currTx, newVal);
+ newOpaqueVal = val.update(currTx, newVal);
}
newVals.add(newOpaqueVal);
}
- _backing.multiPut(keys, newVals);
+ backing.multiPut(keys, newVals);
return ret;
}
@@ -93,26 +93,26 @@
@Override
public void beginCommit(Long txid) {
- _currTx = txid;
- _backing.reset();
+ currTx = txid;
+ backing.reset();
}
@Override
public void commit(Long txid) {
- _currTx = null;
- _backing.reset();
+ currTx = null;
+ backing.reset();
}
static class ReplaceUpdater<T> implements ValueUpdater<T> {
- T _t;
+ T value;
public ReplaceUpdater(T t) {
- _t = t;
+ value = t;
}
@Override
public T update(Object stored) {
- return _t;
+ return value;
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java
index 4f115c0..7b144ca 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/SnapshottableMap.java
@@ -19,53 +19,53 @@
public class SnapshottableMap<T> implements MapState<T>, Snapshottable<T> {
- MapState<T> _delegate;
- List<List<Object>> _keys;
+ MapState<T> delegate;
+ List<List<Object>> keys;
public SnapshottableMap(MapState<T> delegate, List<Object> snapshotKey) {
- _delegate = delegate;
- _keys = Arrays.asList(snapshotKey);
+ this.delegate = delegate;
+ keys = Arrays.asList(snapshotKey);
}
@Override
public List<T> multiGet(List<List<Object>> keys) {
- return _delegate.multiGet(keys);
+ return delegate.multiGet(keys);
}
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- return _delegate.multiUpdate(keys, updaters);
+ return delegate.multiUpdate(keys, updaters);
}
@Override
public void multiPut(List<List<Object>> keys, List<T> vals) {
- _delegate.multiPut(keys, vals);
+ delegate.multiPut(keys, vals);
}
@Override
public void beginCommit(Long txid) {
- _delegate.beginCommit(txid);
+ delegate.beginCommit(txid);
}
@Override
public void commit(Long txid) {
- _delegate.commit(txid);
+ delegate.commit(txid);
}
@Override
public T get() {
- return multiGet(_keys).get(0);
+ return multiGet(keys).get(0);
}
@Override
public T update(ValueUpdater updater) {
List<ValueUpdater> updaters = Arrays.asList(updater);
- return multiUpdate(_keys, updaters).get(0);
+ return multiUpdate(keys, updaters).get(0);
}
@Override
public void set(T o) {
- multiPut(_keys, Arrays.asList(o));
+ multiPut(keys, Arrays.asList(o));
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/state/map/TransactionalMap.java b/storm-client/src/jvm/org/apache/storm/trident/state/map/TransactionalMap.java
index 8923278..ef1a08c 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/state/map/TransactionalMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/state/map/TransactionalMap.java
@@ -19,11 +19,11 @@
public class TransactionalMap<T> implements MapState<T> {
- CachedBatchReadsMap<TransactionalValue> _backing;
- Long _currTx;
+ CachedBatchReadsMap<TransactionalValue> backing;
+ Long currTx;
protected TransactionalMap(IBackingMap<TransactionalValue> backing) {
- _backing = new CachedBatchReadsMap(backing);
+ this.backing = new CachedBatchReadsMap(backing);
}
public static <T> MapState<T> build(IBackingMap<TransactionalValue> backing) {
@@ -32,7 +32,7 @@
@Override
public List<T> multiGet(List<List<Object>> keys) {
- List<CachedBatchReadsMap.RetVal<TransactionalValue>> vals = _backing.multiGet(keys);
+ List<CachedBatchReadsMap.RetVal<TransactionalValue>> vals = backing.multiGet(keys);
List<T> ret = new ArrayList<T>(vals.size());
for (CachedBatchReadsMap.RetVal<TransactionalValue> retval : vals) {
TransactionalValue v = retval.val;
@@ -47,7 +47,7 @@
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- List<CachedBatchReadsMap.RetVal<TransactionalValue>> curr = _backing.multiGet(keys);
+ List<CachedBatchReadsMap.RetVal<TransactionalValue>> curr = backing.multiGet(keys);
List<TransactionalValue> newVals = new ArrayList<TransactionalValue>(curr.size());
List<List<Object>> newKeys = new ArrayList();
List<T> ret = new ArrayList<T>();
@@ -58,13 +58,13 @@
TransactionalValue<T> newVal;
boolean changed = false;
if (val == null) {
- newVal = new TransactionalValue<T>(_currTx, updater.update(null));
+ newVal = new TransactionalValue<T>(currTx, updater.update(null));
changed = true;
} else {
- if (_currTx != null && _currTx.equals(val.getTxid()) && !retval.cached) {
+ if (currTx != null && currTx.equals(val.getTxid()) && !retval.cached) {
newVal = val;
} else {
- newVal = new TransactionalValue<T>(_currTx, updater.update(val.getVal()));
+ newVal = new TransactionalValue<T>(currTx, updater.update(val.getVal()));
changed = true;
}
}
@@ -75,7 +75,7 @@
}
}
if (!newKeys.isEmpty()) {
- _backing.multiPut(newKeys, newVals);
+ backing.multiPut(newKeys, newVals);
}
return ret;
}
@@ -84,20 +84,20 @@
public void multiPut(List<List<Object>> keys, List<T> vals) {
List<TransactionalValue> newVals = new ArrayList<TransactionalValue>(vals.size());
for (T val : vals) {
- newVals.add(new TransactionalValue<T>(_currTx, val));
+ newVals.add(new TransactionalValue<T>(currTx, val));
}
- _backing.multiPut(keys, newVals);
+ backing.multiPut(keys, newVals);
}
@Override
public void beginCommit(Long txid) {
- _currTx = txid;
- _backing.reset();
+ currTx = txid;
+ backing.reset();
}
@Override
public void commit(Long txid) {
- _currTx = null;
- _backing.reset();
+ currTx = null;
+ backing.reset();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/testing/FeederBatchSpout.java b/storm-client/src/jvm/org/apache/storm/trident/testing/FeederBatchSpout.java
index 236ef48..7a65802 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/testing/FeederBatchSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/testing/FeederBatchSpout.java
@@ -29,27 +29,27 @@
public class FeederBatchSpout implements ITridentSpout<Map<Integer, List<List<Object>>>>, IFeeder {
- String _id;
- String _semaphoreId;
- Fields _outFields;
- boolean _waitToEmit = true;
+ String id;
+ String semaphoreId;
+ Fields outFields;
+ boolean waitToEmit = true;
public FeederBatchSpout(List<String> fields) {
- _outFields = new Fields(fields);
- _id = RegisteredGlobalState.registerState(new CopyOnWriteArrayList());
- _semaphoreId = RegisteredGlobalState.registerState(new CopyOnWriteArrayList());
+ outFields = new Fields(fields);
+ id = RegisteredGlobalState.registerState(new CopyOnWriteArrayList());
+ semaphoreId = RegisteredGlobalState.registerState(new CopyOnWriteArrayList());
}
public void setWaitToEmit(boolean trueIfWait) {
- _waitToEmit = trueIfWait;
+ waitToEmit = trueIfWait;
}
@Override
public void feed(Object tuples) {
Semaphore sem = new Semaphore(0);
- ((List) RegisteredGlobalState.getState(_semaphoreId)).add(sem);
- ((List) RegisteredGlobalState.getState(_id)).add(tuples);
+ ((List) RegisteredGlobalState.getState(semaphoreId)).add(sem);
+ ((List) RegisteredGlobalState.getState(id)).add(tuples);
try {
sem.acquire();
} catch (InterruptedException e) {
@@ -64,7 +64,7 @@
@Override
public Fields getOutputFields() {
- return _outFields;
+ return outFields;
}
@Override
@@ -84,15 +84,15 @@
private static class FeederEmitter implements ITridentSpout.Emitter<Map<Integer, List<List<Object>>>> {
- int _index;
+ int index;
public FeederEmitter(int index) {
- _index = index;
+ this.index = index;
}
@Override
public void emitBatch(TransactionAttempt tx, Map<Integer, List<List<Object>>> coordinatorMeta, TridentCollector collector) {
- List<List<Object>> tuples = coordinatorMeta.get(_index);
+ List<List<Object>> tuples = coordinatorMeta.get(index);
if (tuples != null) {
for (List<Object> t : tuples) {
collector.emit(t);
@@ -111,13 +111,13 @@
public class FeederCoordinator implements ITridentSpout.BatchCoordinator<Map<Integer, List<List<Object>>>> {
- int _numPartitions;
- int _emittedIndex = 0;
+ int numPartitions;
+ int emittedIndex = 0;
Map<Long, Integer> txIndices = new HashMap();
- int _masterEmitted = 0;
+ int masterEmitted = 0;
public FeederCoordinator(int numPartitions) {
- _numPartitions = numPartitions;
+ this.numPartitions = numPartitions;
}
@Override
@@ -126,21 +126,21 @@
if (currMetadata != null) {
return currMetadata;
}
- List allBatches = (List) RegisteredGlobalState.getState(_id);
- if (allBatches.size() > _emittedIndex) {
- Object batchInfo = allBatches.get(_emittedIndex);
- txIndices.put(txid, _emittedIndex);
- _emittedIndex += 1;
+ List allBatches = (List) RegisteredGlobalState.getState(id);
+ if (allBatches.size() > emittedIndex) {
+ Object batchInfo = allBatches.get(emittedIndex);
+ txIndices.put(txid, emittedIndex);
+ emittedIndex += 1;
if (batchInfo instanceof Map) {
return (Map) batchInfo;
} else {
List batchList = (List) batchInfo;
Map<Integer, List<List<Object>>> partitions = new HashMap();
- for (int i = 0; i < _numPartitions; i++) {
+ for (int i = 0; i < numPartitions; i++) {
partitions.put(i, new ArrayList());
}
for (int i = 0; i < batchList.size(); i++) {
- int partition = i % _numPartitions;
+ int partition = i % numPartitions;
partitions.get(partition).add((List) batchList.get(i));
}
return partitions;
@@ -158,19 +158,19 @@
public void success(long txid) {
Integer index = txIndices.get(txid);
if (index != null) {
- Semaphore sem = (Semaphore) ((List) RegisteredGlobalState.getState(_semaphoreId)).get(index);
+ Semaphore sem = (Semaphore) ((List) RegisteredGlobalState.getState(semaphoreId)).get(index);
sem.release();
}
}
@Override
public boolean isReady(long txid) {
- if (!_waitToEmit) {
+ if (!waitToEmit) {
return true;
}
- List allBatches = (List) RegisteredGlobalState.getState(_id);
- if (allBatches.size() > _masterEmitted) {
- _masterEmitted++;
+ List allBatches = (List) RegisteredGlobalState.getState(id);
+ if (allBatches.size() > masterEmitted) {
+ masterEmitted++;
return true;
} else {
Utils.sleep(2);
diff --git a/storm-client/src/jvm/org/apache/storm/trident/testing/FeederCommitterBatchSpout.java b/storm-client/src/jvm/org/apache/storm/trident/testing/FeederCommitterBatchSpout.java
index 8d74406..d65a478 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/testing/FeederCommitterBatchSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/testing/FeederCommitterBatchSpout.java
@@ -24,48 +24,48 @@
public class FeederCommitterBatchSpout implements ICommitterTridentSpout<Map<Integer, List<List<Object>>>>, IFeeder {
- FeederBatchSpout _spout;
+ FeederBatchSpout spout;
public FeederCommitterBatchSpout(List<String> fields) {
- _spout = new FeederBatchSpout(fields);
+ spout = new FeederBatchSpout(fields);
}
public void setWaitToEmit(boolean trueIfWait) {
- _spout.setWaitToEmit(trueIfWait);
+ spout.setWaitToEmit(trueIfWait);
}
@Override
public Emitter getEmitter(String txStateId, Map<String, Object> conf, TopologyContext context) {
- return new CommitterEmitter(_spout.getEmitter(txStateId, conf, context));
+ return new CommitterEmitter(spout.getEmitter(txStateId, conf, context));
}
@Override
public BatchCoordinator<Map<Integer, List<List<Object>>>> getCoordinator(String txStateId, Map<String, Object> conf,
TopologyContext context) {
- return _spout.getCoordinator(txStateId, conf, context);
+ return spout.getCoordinator(txStateId, conf, context);
}
@Override
public Fields getOutputFields() {
- return _spout.getOutputFields();
+ return spout.getOutputFields();
}
@Override
public Map<String, Object> getComponentConfiguration() {
- return _spout.getComponentConfiguration();
+ return spout.getComponentConfiguration();
}
@Override
public void feed(Object tuples) {
- _spout.feed(tuples);
+ spout.feed(tuples);
}
static class CommitterEmitter implements ICommitterTridentSpout.Emitter {
- ITridentSpout.Emitter _emitter;
+ ITridentSpout.Emitter emitter;
public CommitterEmitter(ITridentSpout.Emitter e) {
- _emitter = e;
+ emitter = e;
}
@Override
@@ -74,17 +74,17 @@
@Override
public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) {
- _emitter.emitBatch(tx, coordinatorMeta, collector);
+ emitter.emitBatch(tx, coordinatorMeta, collector);
}
@Override
public void success(TransactionAttempt tx) {
- _emitter.success(tx);
+ emitter.success(tx);
}
@Override
public void close() {
- _emitter.close();
+ emitter.close();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/testing/LRUMemoryMapState.java b/storm-client/src/jvm/org/apache/storm/trident/testing/LRUMemoryMapState.java
index 2b12a72..d1c2ba8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/testing/LRUMemoryMapState.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/testing/LRUMemoryMapState.java
@@ -32,92 +32,94 @@
import org.apache.storm.trident.util.LRUMap;
import org.apache.storm.tuple.Values;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class LRUMemoryMapState<T> implements Snapshottable<T>, ITupleCollection, MapState<T> {
- static ConcurrentHashMap<String, Map<List<Object>, Object>> _dbs = new ConcurrentHashMap<String, Map<List<Object>, Object>>();
- LRUMemoryMapStateBacking<OpaqueValue> _backing;
- SnapshottableMap<T> _delegate;
+ static ConcurrentHashMap<String, Map<List<Object>, Object>> dbs = new ConcurrentHashMap<String, Map<List<Object>, Object>>();
+ LRUMemoryMapStateBacking<OpaqueValue> backing;
+ SnapshottableMap<T> delegate;
public LRUMemoryMapState(int cacheSize, String id) {
- _backing = new LRUMemoryMapStateBacking(cacheSize, id);
- _delegate = new SnapshottableMap(OpaqueMap.build(_backing), new Values("$MEMORY-MAP-STATE-GLOBAL$"));
+ backing = new LRUMemoryMapStateBacking(cacheSize, id);
+ delegate = new SnapshottableMap(OpaqueMap.build(backing), new Values("$MEMORY-MAP-STATE-GLOBAL$"));
}
@Override
public T update(ValueUpdater updater) {
- return _delegate.update(updater);
+ return delegate.update(updater);
}
@Override
public void set(T o) {
- _delegate.set(o);
+ delegate.set(o);
}
@Override
public T get() {
- return _delegate.get();
+ return delegate.get();
}
@Override
public void beginCommit(Long txid) {
- _delegate.beginCommit(txid);
+ delegate.beginCommit(txid);
}
@Override
public void commit(Long txid) {
- _delegate.commit(txid);
+ delegate.commit(txid);
}
@Override
public Iterator<List<Object>> getTuples() {
- return _backing.getTuples();
+ return backing.getTuples();
}
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- return _delegate.multiUpdate(keys, updaters);
+ return delegate.multiUpdate(keys, updaters);
}
@Override
public void multiPut(List<List<Object>> keys, List<T> vals) {
- _delegate.multiPut(keys, vals);
+ delegate.multiPut(keys, vals);
}
@Override
public List<T> multiGet(List<List<Object>> keys) {
- return _delegate.multiGet(keys);
+ return delegate.multiGet(keys);
}
public static class Factory implements StateFactory {
- String _id;
- int _maxSize;
+ String id;
+ int maxSize;
public Factory(int maxSize) {
- _id = UUID.randomUUID().toString();
- _maxSize = maxSize;
+ id = UUID.randomUUID().toString();
+ this.maxSize = maxSize;
}
@Override
public State makeState(Map<String, Object> conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
- return new LRUMemoryMapState(_maxSize, _id + partitionIndex);
+ return new LRUMemoryMapState(maxSize, id + partitionIndex);
}
}
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
static class LRUMemoryMapStateBacking<T> implements IBackingMap<T>, ITupleCollection {
Map<List<Object>, T> db;
Long currTx;
public LRUMemoryMapStateBacking(int cacheSize, String id) {
- if (!_dbs.containsKey(id)) {
- _dbs.put(id, new LRUMap<List<Object>, Object>(cacheSize));
+ if (!dbs.containsKey(id)) {
+ dbs.put(id, new LRUMap<List<Object>, Object>(cacheSize));
}
- this.db = (Map<List<Object>, T>) _dbs.get(id);
+ this.db = (Map<List<Object>, T>) dbs.get(id);
}
public static void clearAll() {
- _dbs.clear();
+ dbs.clear();
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryBackingMap.java b/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryBackingMap.java
index 038ffd3..5dc43f5 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryBackingMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryBackingMap.java
@@ -19,13 +19,13 @@
import org.apache.storm.trident.state.map.IBackingMap;
public class MemoryBackingMap implements IBackingMap<Object> {
- Map _vals = new HashMap();
+ Map vals = new HashMap();
@Override
public List<Object> multiGet(List<List<Object>> keys) {
List ret = new ArrayList();
for (List key : keys) {
- ret.add(_vals.get(key));
+ ret.add(vals.get(key));
}
return ret;
}
@@ -35,7 +35,7 @@
for (int i = 0; i < keys.size(); i++) {
List key = keys.get(i);
Object val = vals.get(i);
- _vals.put(key, val);
+ this.vals.put(key, val);
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java b/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java
index b3aed7b..2647626 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/testing/MemoryMapState.java
@@ -35,65 +35,65 @@
public class MemoryMapState<T> implements Snapshottable<T>, ITupleCollection, MapState<T>, RemovableMapState<T> {
- static ConcurrentHashMap<String, Map<List<Object>, Object>> _dbs = new ConcurrentHashMap<String, Map<List<Object>, Object>>();
- MemoryMapStateBacking<OpaqueValue> _backing;
- SnapshottableMap<T> _delegate;
- List<List<Object>> _removed = new ArrayList();
- Long _currTx = null;
+ static ConcurrentHashMap<String, Map<List<Object>, Object>> dbs = new ConcurrentHashMap<String, Map<List<Object>, Object>>();
+ MemoryMapStateBacking<OpaqueValue> backing;
+ SnapshottableMap<T> delegate;
+ List<List<Object>> removed = new ArrayList();
+ Long currTx = null;
public MemoryMapState(String id) {
- _backing = new MemoryMapStateBacking(id);
- _delegate = new SnapshottableMap(OpaqueMap.build(_backing), new Values("$MEMORY-MAP-STATE-GLOBAL$"));
+ backing = new MemoryMapStateBacking(id);
+ delegate = new SnapshottableMap(OpaqueMap.build(backing), new Values("$MEMORY-MAP-STATE-GLOBAL$"));
}
@Override
public T update(ValueUpdater updater) {
- return _delegate.update(updater);
+ return delegate.update(updater);
}
@Override
public void set(T o) {
- _delegate.set(o);
+ delegate.set(o);
}
@Override
public T get() {
- return _delegate.get();
+ return delegate.get();
}
@Override
public void beginCommit(Long txid) {
- _delegate.beginCommit(txid);
- if (txid == null || !txid.equals(_currTx)) {
- _backing.multiRemove(_removed);
+ delegate.beginCommit(txid);
+ if (txid == null || !txid.equals(currTx)) {
+ backing.multiRemove(removed);
}
- _removed = new ArrayList();
- _currTx = txid;
+ removed = new ArrayList();
+ currTx = txid;
}
@Override
public void commit(Long txid) {
- _delegate.commit(txid);
+ delegate.commit(txid);
}
@Override
public Iterator<List<Object>> getTuples() {
- return _backing.getTuples();
+ return backing.getTuples();
}
@Override
public List<T> multiUpdate(List<List<Object>> keys, List<ValueUpdater> updaters) {
- return _delegate.multiUpdate(keys, updaters);
+ return delegate.multiUpdate(keys, updaters);
}
@Override
public void multiPut(List<List<Object>> keys, List<T> vals) {
- _delegate.multiPut(keys, vals);
+ delegate.multiPut(keys, vals);
}
@Override
public List<T> multiGet(List<List<Object>> keys) {
- return _delegate.multiGet(keys);
+ return delegate.multiGet(keys);
}
@Override
@@ -105,20 +105,20 @@
// first just set the keys to null, then flag to remove them at beginning of next commit when we know the current and last value
// are both null
multiPut(keys, nulls);
- _removed.addAll(keys);
+ removed.addAll(keys);
}
public static class Factory implements StateFactory {
- String _id;
+ String id;
public Factory() {
- _id = UUID.randomUUID().toString();
+ id = UUID.randomUUID().toString();
}
@Override
public State makeState(Map<String, Object> conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
- return new MemoryMapState(_id + partitionIndex);
+ return new MemoryMapState(id + partitionIndex);
}
}
@@ -128,14 +128,14 @@
Long currTx;
public MemoryMapStateBacking(String id) {
- if (!_dbs.containsKey(id)) {
- _dbs.put(id, new HashMap());
+ if (!dbs.containsKey(id)) {
+ dbs.put(id, new HashMap());
}
- this.db = (Map<List<Object>, T>) _dbs.get(id);
+ this.db = (Map<List<Object>, T>) dbs.get(id);
}
public static void clearAll() {
- _dbs.clear();
+ dbs.clear();
}
public void multiRemove(List<List<Object>> keys) {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/MasterBatchCoordinator.java b/storm-client/src/jvm/org/apache/storm/trident/topology/MasterBatchCoordinator.java
index 5a9917f..3c5bc7f 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/MasterBatchCoordinator.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/MasterBatchCoordinator.java
@@ -43,69 +43,69 @@
private static final String CURRENT_TX = "currtx";
private static final String CURRENT_ATTEMPTS = "currattempts";
- TreeMap<Long, TransactionStatus> _activeTx = new TreeMap<Long, TransactionStatus>();
- TreeMap<Long, Integer> _attemptIds;
- Long _currTransaction;
- int _maxTransactionActive;
- List<ITridentSpout.BatchCoordinator> _coordinators = new ArrayList();
- List<String> _managedSpoutIds;
- List<ITridentSpout> _spouts;
- WindowedTimeThrottler _throttler;
- boolean _active = true;
- private List<TransactionalState> _states = new ArrayList();
- private SpoutOutputCollector _collector;
+ TreeMap<Long, TransactionStatus> activeTx = new TreeMap<Long, TransactionStatus>();
+ TreeMap<Long, Integer> attemptIds;
+ Long currTransaction;
+ int maxTransactionActive;
+ List<ITridentSpout.BatchCoordinator> coordinators = new ArrayList();
+ List<String> managedSpoutIds;
+ List<ITridentSpout> spouts;
+ WindowedTimeThrottler throttler;
+ boolean active = true;
+ private List<TransactionalState> states = new ArrayList();
+ private SpoutOutputCollector collector;
public MasterBatchCoordinator(List<String> spoutIds, List<ITridentSpout> spouts) {
if (spoutIds.isEmpty()) {
throw new IllegalArgumentException("Must manage at least one spout");
}
- _managedSpoutIds = spoutIds;
- _spouts = spouts;
+ managedSpoutIds = spoutIds;
+ this.spouts = spouts;
LOG.debug("Created {}", this);
}
public List<String> getManagedSpoutIds() {
- return _managedSpoutIds;
+ return managedSpoutIds;
}
@Override
public void activate() {
- _active = true;
+ active = true;
}
@Override
public void deactivate() {
- _active = false;
+ active = false;
}
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
- _throttler = new WindowedTimeThrottler((Number) conf.get(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS), 1);
- for (String spoutId : _managedSpoutIds) {
- _states.add(TransactionalState.newCoordinatorState(conf, spoutId));
+ throttler = new WindowedTimeThrottler((Number) conf.get(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS), 1);
+ for (String spoutId : managedSpoutIds) {
+ states.add(TransactionalState.newCoordinatorState(conf, spoutId));
}
- _currTransaction = getStoredCurrTransaction();
+ currTransaction = getStoredCurrTransaction();
- _collector = collector;
+ this.collector = collector;
Number active = (Number) conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING);
if (active == null) {
- _maxTransactionActive = 1;
+ maxTransactionActive = 1;
} else {
- _maxTransactionActive = active.intValue();
+ maxTransactionActive = active.intValue();
}
- _attemptIds = getStoredCurrAttempts(_currTransaction, _maxTransactionActive);
+ attemptIds = getStoredCurrAttempts(currTransaction, maxTransactionActive);
- for (int i = 0; i < _spouts.size(); i++) {
- String txId = _managedSpoutIds.get(i);
- _coordinators.add(_spouts.get(i).getCoordinator(txId, conf, context));
+ for (int i = 0; i < spouts.size(); i++) {
+ String txId = managedSpoutIds.get(i);
+ coordinators.add(spouts.get(i).getCoordinator(txId, conf, context));
}
LOG.debug("Opened {}", this);
}
@Override
public void close() {
- for (TransactionalState state : _states) {
+ for (TransactionalState state : states) {
state.close();
}
LOG.debug("Closed {}", this);
@@ -119,19 +119,19 @@
@Override
public void ack(Object msgId) {
TransactionAttempt tx = (TransactionAttempt) msgId;
- TransactionStatus status = _activeTx.get(tx.getTransactionId());
+ TransactionStatus status = activeTx.get(tx.getTransactionId());
LOG.debug("Ack. [tx_attempt = {}], [tx_status = {}], [{}]", tx, status, this);
if (status != null && tx.equals(status.attempt)) {
if (status.status == AttemptStatus.PROCESSING) {
status.status = AttemptStatus.PROCESSED;
LOG.debug("Changed status. [tx_attempt = {}] [tx_status = {}]", tx, status);
} else if (status.status == AttemptStatus.COMMITTING) {
- _activeTx.remove(tx.getTransactionId());
- _attemptIds.remove(tx.getTransactionId());
- _collector.emit(SUCCESS_STREAM_ID, new Values(tx));
- _currTransaction = nextTransactionId(tx.getTransactionId());
- for (TransactionalState state : _states) {
- state.setData(CURRENT_TX, _currTransaction);
+ activeTx.remove(tx.getTransactionId());
+ attemptIds.remove(tx.getTransactionId());
+ collector.emit(SUCCESS_STREAM_ID, new Values(tx));
+ currTransaction = nextTransactionId(tx.getTransactionId());
+ for (TransactionalState state : states) {
+ state.setData(CURRENT_TX, currTransaction);
}
LOG.debug("Emitted on [stream = {}], [tx_attempt = {}], [tx_status = {}], [{}]", SUCCESS_STREAM_ID, tx, status, this);
}
@@ -142,10 +142,10 @@
@Override
public void fail(Object msgId) {
TransactionAttempt tx = (TransactionAttempt) msgId;
- TransactionStatus stored = _activeTx.remove(tx.getTransactionId());
+ TransactionStatus stored = activeTx.remove(tx.getTransactionId());
LOG.debug("Fail. [tx_attempt = {}], [tx_status = {}], [{}]", tx, stored, this);
if (stored != null && tx.equals(stored.attempt)) {
- _activeTx.tailMap(tx.getTransactionId()).clear();
+ activeTx.tailMap(tx.getTransactionId()).clear();
sync();
}
}
@@ -164,39 +164,39 @@
// max_spout_pending = 3
// tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet),
// and there won't be a batch for tx 4 because there's max_spout_pending tx active
- TransactionStatus maybeCommit = _activeTx.get(_currTransaction);
+ TransactionStatus maybeCommit = activeTx.get(currTransaction);
if (maybeCommit != null && maybeCommit.status == AttemptStatus.PROCESSED) {
maybeCommit.status = AttemptStatus.COMMITTING;
- _collector.emit(COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt);
+ collector.emit(COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt);
LOG.debug("Emitted on [stream = {}], [tx_status = {}], [{}]", COMMIT_STREAM_ID, maybeCommit, this);
}
- if (_active) {
- if (_activeTx.size() < _maxTransactionActive) {
- Long curr = _currTransaction;
- for (int i = 0; i < _maxTransactionActive; i++) {
- if (!_activeTx.containsKey(curr) && isReady(curr)) {
+ if (active) {
+ if (activeTx.size() < maxTransactionActive) {
+ Long curr = currTransaction;
+ for (int i = 0; i < maxTransactionActive; i++) {
+ if (!activeTx.containsKey(curr) && isReady(curr)) {
// by using a monotonically increasing attempt id, downstream tasks
// can be memory efficient by clearing out state for old attempts
// as soon as they see a higher attempt id for a transaction
- Integer attemptId = _attemptIds.get(curr);
+ Integer attemptId = attemptIds.get(curr);
if (attemptId == null) {
attemptId = 0;
} else {
attemptId++;
}
- _attemptIds.put(curr, attemptId);
- for (TransactionalState state : _states) {
- state.setData(CURRENT_ATTEMPTS, _attemptIds);
+ attemptIds.put(curr, attemptId);
+ for (TransactionalState state : states) {
+ state.setData(CURRENT_ATTEMPTS, attemptIds);
}
TransactionAttempt attempt = new TransactionAttempt(curr, attemptId);
final TransactionStatus newTransactionStatus = new TransactionStatus(attempt);
- _activeTx.put(curr, newTransactionStatus);
- _collector.emit(BATCH_STREAM_ID, new Values(attempt), attempt);
+ activeTx.put(curr, newTransactionStatus);
+ collector.emit(BATCH_STREAM_ID, new Values(attempt), attempt);
LOG.debug("Emitted on [stream = {}], [tx_attempt = {}], [tx_status = {}], [{}]", BATCH_STREAM_ID, attempt,
newTransactionStatus, this);
- _throttler.markEvent();
+ throttler.markEvent();
}
curr = nextTransactionId(curr);
}
@@ -205,11 +205,11 @@
}
private boolean isReady(long txid) {
- if (_throttler.isThrottled()) {
+ if (throttler.isThrottled()) {
return false;
}
//TODO: make this strategy configurable?... right now it goes if anyone is ready
- for (ITridentSpout.BatchCoordinator coord : _coordinators) {
+ for (ITridentSpout.BatchCoordinator coord : coordinators) {
if (coord.isReady(txid)) {
return true;
}
@@ -231,7 +231,7 @@
private Long getStoredCurrTransaction() {
Long ret = INIT_TXID;
- for (TransactionalState state : _states) {
+ for (TransactionalState state : states) {
Long curr = (Long) state.getData(CURRENT_TX);
if (curr != null && curr.compareTo(ret) > 0) {
ret = curr;
@@ -242,7 +242,7 @@
private TreeMap<Long, Integer> getStoredCurrAttempts(long currTransaction, int maxBatches) {
TreeMap<Long, Integer> ret = new TreeMap<Long, Integer>();
- for (TransactionalState state : _states) {
+ for (TransactionalState state : states) {
Map<Object, Number> attempts = (Map) state.getData(CURRENT_ATTEMPTS);
if (attempts == null) {
attempts = new HashMap();
@@ -271,19 +271,19 @@
@Override
public String toString() {
- return "MasterBatchCoordinator{" +
- "_states=" + _states +
- ", _activeTx=" + _activeTx +
- ", _attemptIds=" + _attemptIds +
- ", _collector=" + _collector +
- ", _currTransaction=" + _currTransaction +
- ", _maxTransactionActive=" + _maxTransactionActive +
- ", _coordinators=" + _coordinators +
- ", _managedSpoutIds=" + _managedSpoutIds +
- ", _spouts=" + _spouts +
- ", _throttler=" + _throttler +
- ", _active=" + _active +
- "}";
+ return "MasterBatchCoordinator{"
+ + "states=" + states
+ + ", activeTx=" + activeTx
+ + ", attemptIds=" + attemptIds
+ + ", collector=" + collector
+ + ", currTransaction=" + currTransaction
+ + ", maxTransactionActive=" + maxTransactionActive
+ + ", coordinators=" + coordinators
+ + ", managedSpoutIds=" + managedSpoutIds
+ + ", spouts=" + spouts
+ + ", throttler=" + throttler
+ + ", active=" + active
+ + "}";
}
private static enum AttemptStatus {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/TransactionAttempt.java b/storm-client/src/jvm/org/apache/storm/trident/topology/TransactionAttempt.java
index 1f90b38..710599e 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/TransactionAttempt.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/TransactionAttempt.java
@@ -16,8 +16,8 @@
public class TransactionAttempt implements IBatchID {
- Long _txid;
- int _attemptId;
+ Long txid;
+ int attemptId;
// for kryo compatibility
@@ -26,27 +26,27 @@
}
public TransactionAttempt(Long txid, int attemptId) {
- _txid = txid;
- _attemptId = attemptId;
+ this.txid = txid;
+ this.attemptId = attemptId;
}
public Long getTransactionId() {
- return _txid;
+ return txid;
}
@Override
public Object getId() {
- return _txid;
+ return txid;
}
@Override
public int getAttemptId() {
- return _attemptId;
+ return attemptId;
}
@Override
public int hashCode() {
- return _txid.hashCode();
+ return txid.hashCode();
}
@Override
@@ -55,11 +55,11 @@
return false;
}
TransactionAttempt other = (TransactionAttempt) o;
- return _txid.equals(other._txid) && _attemptId == other._attemptId;
+ return txid.equals(other.txid) && attemptId == other.attemptId;
}
@Override
public String toString() {
- return "" + _txid + ":" + _attemptId;
+ return "" + txid + ":" + attemptId;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java b/storm-client/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java
index e3500b2..7a187c9 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/TridentBoltExecutor.java
@@ -43,45 +43,45 @@
public class TridentBoltExecutor implements IRichBolt {
public static final String COORD_STREAM_PREFIX = "$coord-";
- Map<GlobalStreamId, String> _batchGroupIds;
- Map<String, CoordSpec> _coordSpecs;
- Map<String, CoordCondition> _coordConditions;
- ITridentBatchBolt _bolt;
- long _messageTimeoutMs;
- long _lastRotate;
- RotatingMap<Object, TrackedBatch> _batches;
- OutputCollector _collector;
- CoordinatedOutputCollector _coordCollector;
- BatchOutputCollector _coordOutputCollector;
- TopologyContext _context;
+ Map<GlobalStreamId, String> batchGroupIds;
+ Map<String, CoordSpec> coordSpecs;
+ Map<String, CoordCondition> coordConditions;
+ ITridentBatchBolt bolt;
+ long messageTimeoutMs;
+ long lastRotate;
+ RotatingMap<Object, TrackedBatch> batches;
+ OutputCollector collector;
+ CoordinatedOutputCollector coordCollector;
+ BatchOutputCollector coordOutputCollector;
+ TopologyContext context;
// map from batchgroupid to coordspec
public TridentBoltExecutor(ITridentBatchBolt bolt, Map<GlobalStreamId, String> batchGroupIds,
Map<String, CoordSpec> coordinationSpecs) {
- _batchGroupIds = batchGroupIds;
- _coordSpecs = coordinationSpecs;
- _bolt = bolt;
+ this.batchGroupIds = batchGroupIds;
+ coordSpecs = coordinationSpecs;
+ this.bolt = bolt;
}
- public static String COORD_STREAM(String batch) {
+ public static String coordStream(String batch) {
return COORD_STREAM_PREFIX + batch;
}
@Override
public void prepare(Map<String, Object> conf, TopologyContext context, OutputCollector collector) {
- _messageTimeoutMs = context.maxTopologyMessageTimeout() * 1000L;
- _lastRotate = System.currentTimeMillis();
- _batches = new RotatingMap<>(2);
- _context = context;
- _collector = collector;
- _coordCollector = new CoordinatedOutputCollector(collector);
- _coordOutputCollector = new BatchOutputCollectorImpl(new OutputCollector(_coordCollector));
+ messageTimeoutMs = context.maxTopologyMessageTimeout() * 1000L;
+ lastRotate = System.currentTimeMillis();
+ batches = new RotatingMap<>(2);
+ this.context = context;
+ this.collector = collector;
+ coordCollector = new CoordinatedOutputCollector(collector);
+ coordOutputCollector = new BatchOutputCollectorImpl(new OutputCollector(coordCollector));
- _coordConditions = (Map) context.getExecutorData("__coordConditions");
- if (_coordConditions == null) {
- _coordConditions = new HashMap<>();
- for (String batchGroup : _coordSpecs.keySet()) {
- CoordSpec spec = _coordSpecs.get(batchGroup);
+ coordConditions = (Map) context.getExecutorData("__coordConditions");
+ if (coordConditions == null) {
+ coordConditions = new HashMap<>();
+ for (String batchGroup : coordSpecs.keySet()) {
+ CoordSpec spec = coordSpecs.get(batchGroup);
CoordCondition cond = new CoordCondition();
cond.commitStream = spec.commitStream;
cond.expectedTaskReports = 0;
@@ -95,24 +95,24 @@
}
cond.targetTasks = new HashSet<>();
for (String component : Utils.get(context.getThisTargets(),
- COORD_STREAM(batchGroup),
+ coordStream(batchGroup),
new HashMap<String, Grouping>()).keySet()) {
cond.targetTasks.addAll(context.getComponentTasks(component));
}
- _coordConditions.put(batchGroup, cond);
+ coordConditions.put(batchGroup, cond);
}
- context.setExecutorData("_coordConditions", _coordConditions);
+ context.setExecutorData("coordConditions", coordConditions);
}
- _bolt.prepare(conf, context, _coordOutputCollector);
+ bolt.prepare(conf, context, coordOutputCollector);
}
private void failBatch(TrackedBatch tracked, FailedException e) {
if (e != null && e instanceof ReportedFailedException) {
- _collector.reportError(e);
+ collector.reportError(e);
}
tracked.failed = true;
if (tracked.delayedAck != null) {
- _collector.fail(tracked.delayedAck);
+ collector.fail(tracked.delayedAck);
tracked.delayedAck = null;
}
}
@@ -124,33 +124,33 @@
private boolean finishBatch(TrackedBatch tracked, Tuple finishTuple) {
boolean success = true;
try {
- _bolt.finishBatch(tracked.info);
- String stream = COORD_STREAM(tracked.info.batchGroup);
+ bolt.finishBatch(tracked.info);
+ String stream = coordStream(tracked.info.batchGroup);
for (Integer task : tracked.condition.targetTasks) {
- _collector
+ collector
.emitDirect(task, stream, finishTuple, new Values(tracked.info.batchId, Utils.get(tracked.taskEmittedTuples, task, 0)));
}
if (tracked.delayedAck != null) {
- _collector.ack(tracked.delayedAck);
+ collector.ack(tracked.delayedAck);
tracked.delayedAck = null;
}
} catch (FailedException e) {
failBatch(tracked, e);
success = false;
}
- _batches.remove(tracked.info.batchId.getId());
+ batches.remove(tracked.info.batchId.getId());
return success;
}
private void checkFinish(TrackedBatch tracked, Tuple tuple, TupleType type) {
if (tracked.failed) {
failBatch(tracked);
- _collector.fail(tuple);
+ collector.fail(tuple);
return;
}
CoordCondition cond = tracked.condition;
- boolean delayed = tracked.delayedAck == null &&
- (cond.commitStream != null && type == TupleType.COMMIT
+ boolean delayed = tracked.delayedAck == null
+ && (cond.commitStream != null && type == TupleType.COMMIT
|| cond.commitStream == null);
if (delayed) {
tracked.delayedAck = tuple;
@@ -162,13 +162,13 @@
} else {
//TODO: add logging that not all tuples were received
failBatch(tracked);
- _collector.fail(tuple);
+ collector.fail(tuple);
failed = true;
}
}
if (!delayed && !failed) {
- _collector.ack(tuple);
+ collector.ack(tuple);
}
}
@@ -177,43 +177,31 @@
public void execute(Tuple tuple) {
if (TupleUtils.isTick(tuple)) {
long now = System.currentTimeMillis();
- if (now - _lastRotate > _messageTimeoutMs) {
- _batches.rotate();
- _lastRotate = now;
+ if (now - lastRotate > messageTimeoutMs) {
+ batches.rotate();
+ lastRotate = now;
}
return;
}
- String batchGroup = _batchGroupIds.get(tuple.getSourceGlobalStreamId());
+ String batchGroup = batchGroupIds.get(tuple.getSourceGlobalStreamId());
if (batchGroup == null) {
// this is so we can do things like have simple DRPC that doesn't need to use batch processing
- _coordCollector.setCurrBatch(null);
- _bolt.execute(null, tuple);
- _collector.ack(tuple);
+ coordCollector.setCurrBatch(null);
+ bolt.execute(null, tuple);
+ collector.ack(tuple);
return;
}
IBatchID id = (IBatchID) tuple.getValue(0);
//get transaction id
//if it already exists and attempt id is greater than the attempt there
-
- TrackedBatch tracked = (TrackedBatch) _batches.get(id.getId());
- // if(_batches.size() > 10 && _context.getThisTaskIndex() == 0) {
- // System.out.println("Received in " + _context.getThisComponentId() + " " + _context.getThisTaskIndex()
- // + " (" + _batches.size() + ")" +
- // "\ntuple: " + tuple +
- // "\nwith tracked " + tracked +
- // "\nwith id " + id +
- // "\nwith group " + batchGroup
- // + "\n");
- //
- // }
- //System.out.println("Num tracked: " + _batches.size() + " " + _context.getThisComponentId() + " " + _context.getThisTaskIndex());
+ TrackedBatch tracked = (TrackedBatch) batches.get(id.getId());
// this code here ensures that only one attempt is ever tracked for a batch, so when
// failures happen you don't get an explosion in memory usage in the tasks
if (tracked != null) {
if (id.getAttemptId() > tracked.attemptId) {
- _batches.remove(id.getId());
+ batches.remove(id.getId());
tracked = null;
} else if (id.getAttemptId() < tracked.attemptId) {
// no reason to try to execute a previous attempt than we've already seen
@@ -223,11 +211,11 @@
if (tracked == null) {
tracked =
- new TrackedBatch(new BatchInfo(batchGroup, id, _bolt.initBatchState(batchGroup, id)), _coordConditions.get(batchGroup),
+ new TrackedBatch(new BatchInfo(batchGroup, id, bolt.initBatchState(batchGroup, id)), coordConditions.get(batchGroup),
id.getAttemptId());
- _batches.put(id.getId(), tracked);
+ batches.put(id.getId(), tracked);
}
- _coordCollector.setCurrBatch(tracked);
+ coordCollector.setCurrBatch(tracked);
//System.out.println("TRACKED: " + tracked + " " + tuple);
@@ -244,7 +232,7 @@
tracked.receivedTuples++;
boolean success = true;
try {
- _bolt.execute(tracked.info, tuple);
+ bolt.execute(tracked.info, tuple);
if (tracked.condition.expectedTaskReports == 0) {
success = finishBatch(tracked, tuple);
}
@@ -252,30 +240,30 @@
failBatch(tracked, e);
}
if (success) {
- _collector.ack(tuple);
+ collector.ack(tuple);
} else {
- _collector.fail(tuple);
+ collector.fail(tuple);
}
}
- _coordCollector.setCurrBatch(null);
+ coordCollector.setCurrBatch(null);
}
@Override
public void cleanup() {
- _bolt.cleanup();
+ bolt.cleanup();
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
- _bolt.declareOutputFields(declarer);
- for (String batchGroup : _coordSpecs.keySet()) {
- declarer.declareStream(COORD_STREAM(batchGroup), true, new Fields("id", "count"));
+ bolt.declareOutputFields(declarer);
+ for (String batchGroup : coordSpecs.keySet()) {
+ declarer.declareStream(coordStream(batchGroup), true, new Fields("id", "count"));
}
}
@Override
public Map<String, Object> getComponentConfiguration() {
- Map<String, Object> ret = _bolt.getComponentConfiguration();
+ Map<String, Object> ret = bolt.getComponentConfiguration();
if (ret == null) {
ret = new HashMap<>();
}
@@ -388,21 +376,21 @@
}
private static class CoordinatedOutputCollector implements IOutputCollector {
- IOutputCollector _delegate;
+ IOutputCollector delegate;
- TrackedBatch _currBatch = null;
+ TrackedBatch currBatch = null;
public CoordinatedOutputCollector(IOutputCollector delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
public void setCurrBatch(TrackedBatch batch) {
- _currBatch = batch;
+ currBatch = batch;
}
@Override
public List<Integer> emit(String stream, Collection<Tuple> anchors, List<Object> tuple) {
- List<Integer> tasks = _delegate.emit(stream, anchors, tuple);
+ List<Integer> tasks = delegate.emit(stream, anchors, tuple);
updateTaskCounts(tasks);
return tasks;
}
@@ -410,7 +398,7 @@
@Override
public void emitDirect(int task, String stream, Collection<Tuple> anchors, List<Object> tuple) {
updateTaskCounts(Arrays.asList(task));
- _delegate.emitDirect(task, stream, anchors, tuple);
+ delegate.emitDirect(task, stream, anchors, tuple);
}
@Override
@@ -430,18 +418,18 @@
@Override
public void flush() {
- _delegate.flush();
+ delegate.flush();
}
@Override
public void reportError(Throwable error) {
- _delegate.reportError(error);
+ delegate.reportError(error);
}
private void updateTaskCounts(List<Integer> tasks) {
- if (_currBatch != null) {
- Map<Integer, Integer> taskEmittedTuples = _currBatch.taskEmittedTuples;
+ if (currBatch != null) {
+ Map<Integer, Integer> taskEmittedTuples = currBatch.taskEmittedTuples;
for (Integer task : tasks) {
int newCount = Utils.get(taskEmittedTuples, task, 0) + 1;
taskEmittedTuples.put(task, newCount);
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/TridentTopologyBuilder.java b/storm-client/src/jvm/org/apache/storm/trident/topology/TridentTopologyBuilder.java
index bbe5d7e..f433590 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/TridentTopologyBuilder.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/TridentTopologyBuilder.java
@@ -49,7 +49,7 @@
static final String SPOUT_COORD_PREFIX = "$spoutcoord-";
Map<GlobalStreamId, String> batchIds = new HashMap<>();
Map<String, TransactionalSpoutComponent> spouts = new HashMap<>();
- Map<String, SpoutComponent> _batchPerTupleSpouts = new HashMap<>();
+ Map<String, SpoutComponent> batchPerTupleSpouts = new HashMap<>();
Map<String, Component> bolts = new HashMap<>();
public static String spoutCoordinator(String spoutId) {
@@ -65,7 +65,7 @@
batchGroups.put(streamName, batchGroup);
markBatchGroups(id, batchGroups);
SpoutComponent c = new SpoutComponent(spout, streamName, parallelism, batchGroup);
- _batchPerTupleSpouts.put(id, c);
+ batchPerTupleSpouts.put(id, c);
return new SpoutDeclarerImpl(c);
}
@@ -121,7 +121,7 @@
//this takes care of setting up coord streams for spouts and bolts
for (GlobalStreamId s : batchIds.keySet()) {
String b = batchIds.get(s);
- ret.put(new GlobalStreamId(s.get_componentId(), TridentBoltExecutor.COORD_STREAM(b)), b);
+ ret.put(new GlobalStreamId(s.get_componentId(), TridentBoltExecutor.coordStream(b)), b);
}
return ret;
@@ -183,8 +183,8 @@
}
}
- for (String id : _batchPerTupleSpouts.keySet()) {
- SpoutComponent c = _batchPerTupleSpouts.get(id);
+ for (String id : batchPerTupleSpouts.keySet()) {
+ SpoutComponent c = batchPerTupleSpouts.get(id);
SpoutDeclarer d =
builder.setSpout(id, new RichSpoutBatchTriggerer((IRichSpout) c.spout, c.streamName, c.batchGroupId), c.parallelism);
@@ -225,7 +225,7 @@
}
CoordSpec spec = specs.get(batch);
CoordType ct;
- if (_batchPerTupleSpouts.containsKey(s.get_componentId())) {
+ if (batchPerTupleSpouts.containsKey(s.get_componentId())) {
ct = CoordType.single();
} else {
ct = CoordType.all();
@@ -250,7 +250,7 @@
Map<String, Set<String>> batchToComponents = getBoltBatchToComponentSubscriptions(id);
for (Map.Entry<String, Set<String>> entry : batchToComponents.entrySet()) {
for (String comp : entry.getValue()) {
- d.directGrouping(comp, TridentBoltExecutor.COORD_STREAM(entry.getKey()));
+ d.directGrouping(comp, TridentBoltExecutor.coordStream(entry.getKey()));
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/state/RotatingTransactionalState.java b/storm-client/src/jvm/org/apache/storm/trident/topology/state/RotatingTransactionalState.java
index 81db7fa..52b7f05 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/state/RotatingTransactionalState.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/state/RotatingTransactionalState.java
@@ -23,23 +23,23 @@
public class RotatingTransactionalState {
private static final Logger LOG = LoggerFactory.getLogger(RotatingTransactionalState.class);
- private TransactionalState _state;
- private String _subdir;
- private TreeMap<Long, Object> _curr = new TreeMap<Long, Object>();
+ private TransactionalState state;
+ private String subdir;
+ private TreeMap<Long, Object> curr = new TreeMap<Long, Object>();
public RotatingTransactionalState(TransactionalState state, String subdir) {
- _state = state;
- _subdir = subdir;
+ this.state = state;
+ this.subdir = subdir;
state.mkdir(subdir);
sync();
LOG.debug("Created {}", this);
}
public Object getLastState() {
- if (_curr.isEmpty()) {
+ if (curr.isEmpty()) {
return null;
} else {
- return _curr.lastEntry().getValue();
+ return curr.lastEntry().getValue();
}
}
@@ -47,33 +47,33 @@
LOG.debug("Overriding state. [txid = {}], [state = {}]", txid, state);
LOG.trace("[{}]", this);
- _state.setData(txPath(txid), state);
- _curr.put(txid, state);
+ this.state.setData(txPath(txid), state);
+ curr.put(txid, state);
LOG.trace("Overriding state complete. [{}]", this);
}
public void removeState(long txid) {
Object state = null;
- if (_curr.containsKey(txid)) {
- state = _curr.remove(txid);
- _state.delete(txPath(txid));
+ if (curr.containsKey(txid)) {
+ state = curr.remove(txid);
+ this.state.delete(txPath(txid));
}
LOG.debug("Removed [state = {}], [txid = {}]", state, txid);
LOG.trace("[{}]", this);
}
public Object getState(long txid) {
- final Object state = _curr.get(txid);
+ final Object state = curr.get(txid);
LOG.debug("Getting state. [txid = {}] => [state = {}]", txid, state);
LOG.trace("Internal state [{}]", this);
return state;
}
public Object getState(long txid, StateInitializer init) {
- if (!_curr.containsKey(txid)) {
- SortedMap<Long, Object> prevMap = _curr.headMap(txid);
- SortedMap<Long, Object> afterMap = _curr.tailMap(txid);
+ if (!curr.containsKey(txid)) {
+ SortedMap<Long, Object> prevMap = curr.headMap(txid);
+ SortedMap<Long, Object> afterMap = curr.tailMap(txid);
Long prev = null;
if (!prevMap.isEmpty()) {
@@ -84,7 +84,7 @@
if (afterMap.isEmpty()) {
Object prevData;
if (prev != null) {
- prevData = _curr.get(prev);
+ prevData = curr.get(prev);
} else {
prevData = null;
}
@@ -92,17 +92,17 @@
} else {
data = null;
}
- _curr.put(txid, data);
- _state.setData(txPath(txid), data);
+ curr.put(txid, data);
+ state.setData(txPath(txid), data);
}
- Object state = _curr.get(txid);
+ Object state = curr.get(txid);
LOG.debug("Getting or initializing state. [txid = {}] => [state = {}]", txid, state);
LOG.trace("[{}]", this);
return state;
}
public Object getPreviousState(long txid) {
- final SortedMap<Long, Object> prevMap = _curr.headMap(txid);
+ final SortedMap<Long, Object> prevMap = curr.headMap(txid);
Object state;
if (prevMap.isEmpty()) {
@@ -117,7 +117,7 @@
}
public boolean hasCache(long txid) {
- return _curr.containsKey(txid);
+ return curr.containsKey(txid);
}
/**
@@ -125,8 +125,8 @@
*/
public Object getStateOrCreate(long txid, StateInitializer init) {
Object state;
- if (_curr.containsKey(txid)) {
- state = _curr.get(txid);
+ if (curr.containsKey(txid)) {
+ state = curr.get(txid);
} else {
getState(txid, init);
state = null;
@@ -135,13 +135,13 @@
}
public void cleanupBefore(long txid) {
- SortedMap<Long, Object> toDelete = _curr.headMap(txid);
+ SortedMap<Long, Object> toDelete = curr.headMap(txid);
for (long tx : new HashSet<Long>(toDelete.keySet())) {
- _curr.remove(tx);
+ curr.remove(tx);
try {
- _state.delete(txPath(tx));
+ state.delete(txPath(tx));
} catch (RuntimeException e) {
- // Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since
+ // Ignore NoNodeExists exceptions because when sync() it may populate curr with stale data since
// zookeeper reads are eventually consistent.
if (!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
throw e;
@@ -151,10 +151,10 @@
}
private void sync() {
- List<String> txids = _state.list(_subdir);
- for (String txid_s : txids) {
- Object data = _state.getData(txPath(txid_s));
- _curr.put(Long.parseLong(txid_s), data);
+ List<String> txids = state.list(subdir);
+ for (String txid : txids) {
+ Object data = state.getData(txPath(txid));
+ curr.put(Long.parseLong(txid), data);
}
}
@@ -163,16 +163,16 @@
}
private String txPath(String tx) {
- return _subdir + "/" + tx;
+ return subdir + "/" + tx;
}
@Override
public String toString() {
- return "RotatingTransactionalState{" +
- "_state=" + _state +
- ", _subdir='" + _subdir + '\'' +
- ", _curr=" + _curr +
- '}';
+ return "RotatingTransactionalState{"
+ + "state=" + state
+ + ", subdir='" + subdir + '\''
+ + ", curr=" + curr
+ + '}';
}
public static interface StateInitializer {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/topology/state/TransactionalState.java b/storm-client/src/jvm/org/apache/storm/trident/topology/state/TransactionalState.java
index 41ab0d2..ccb5feb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/topology/state/TransactionalState.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/topology/state/TransactionalState.java
@@ -12,12 +12,12 @@
package org.apache.storm.trident.topology.state;
-
import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+
import org.apache.storm.Config;
import org.apache.storm.cluster.DaemonType;
import org.apache.storm.shade.org.apache.curator.framework.CuratorFramework;
@@ -31,6 +31,7 @@
import org.apache.storm.utils.CuratorUtils;
import org.apache.storm.utils.Utils;
import org.apache.storm.utils.ZookeeperAuthInfo;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -41,8 +42,8 @@
public class TransactionalState {
private static final Logger LOG = LoggerFactory.getLogger(TransactionalState.class);
- CuratorFramework _curator;
- List<ACL> _zkAcls = null;
+ CuratorFramework curator;
+ List<ACL> zkAcls = null;
protected TransactionalState(Map<String, Object> conf, String id, String subroot) {
try {
@@ -54,18 +55,20 @@
Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT);
ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf);
CuratorFramework initter = CuratorUtils.newCuratorStarted(conf, servers, port, auth, DaemonType.WORKER.getDefaultZkAcls(conf));
- _zkAcls = Utils.getWorkerACL(conf);
+ zkAcls = Utils.getWorkerACL(conf);
try {
TransactionalState.createNode(initter, transactionalRoot, null, null, null);
} catch (KeeperException.NodeExistsException e) {
+ //ignore
}
try {
- TransactionalState.createNode(initter, rootDir, null, _zkAcls, null);
+ TransactionalState.createNode(initter, rootDir, null, zkAcls, null);
} catch (KeeperException.NodeExistsException e) {
+ //ignore
}
initter.close();
- _curator = CuratorUtils.newCuratorStarted(conf, servers, port, rootDir, auth, DaemonType.WORKER.getDefaultZkAcls(conf));
+ curator = CuratorUtils.newCuratorStarted(conf, servers, port, rootDir, auth, DaemonType.WORKER.getDefaultZkAcls(conf));
} catch (Exception e) {
throw new RuntimeException(e);
}
@@ -117,10 +120,10 @@
throw new RuntimeException(e);
}
try {
- if (_curator.checkExists().forPath(path) != null) {
- _curator.setData().forPath(path, ser);
+ if (curator.checkExists().forPath(path) != null) {
+ curator.setData().forPath(path, ser);
} else {
- TransactionalState.createNode(_curator, path, ser, _zkAcls,
+ TransactionalState.createNode(curator, path, ser, zkAcls,
CreateMode.PERSISTENT);
}
} catch (KeeperException.NodeExistsException nne) {
@@ -133,7 +136,7 @@
public void delete(String path) {
path = "/" + path;
try {
- _curator.delete().forPath(path);
+ curator.delete().forPath(path);
} catch (KeeperException.NoNodeException nne) {
LOG.warn("Path {} already deleted.");
} catch (Exception e) {
@@ -146,10 +149,10 @@
path = "/" + path;
try {
List<String> children;
- if (_curator.checkExists().forPath(path) == null) {
+ if (curator.checkExists().forPath(path) == null) {
children = new ArrayList<>();
} else {
- children = _curator.getChildren().forPath(path);
+ children = curator.getChildren().forPath(path);
}
LOG.debug("List [path = {}], [children = {}]", path, children);
return children;
@@ -166,11 +169,11 @@
path = "/" + path;
try {
Object data;
- if (_curator.checkExists().forPath(path) != null) {
+ if (curator.checkExists().forPath(path) != null) {
// Use parseWithException instead of parse so we can capture deserialization errors in the log.
// They are likely to be bugs in the spout code.
try {
- data = JSONValue.parseWithException(new String(_curator.getData().forPath(path), "UTF-8"));
+ data = JSONValue.parseWithException(new String(curator.getData().forPath(path), "UTF-8"));
} catch (ParseException e) {
LOG.warn("Failed to deserialize zookeeper data for path {}", path, e);
data = null;
@@ -186,7 +189,7 @@
}
public void close() {
- _curator.close();
+ curator.close();
}
private Object getWithBackup(Map<String, Object> amap, String primary, String backup) {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/tuple/ComboList.java b/storm-client/src/jvm/org/apache/storm/trident/tuple/ComboList.java
index 24cbb67..3ab06a8 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/tuple/ComboList.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/tuple/ComboList.java
@@ -18,23 +18,23 @@
import org.apache.storm.shade.org.apache.commons.lang.builder.ToStringBuilder;
public class ComboList extends AbstractList<Object> {
- Pointer[] _index;
- List[] _delegates;
+ Pointer[] index;
+ List[] delegates;
public ComboList(List[] delegates, Pointer[] index) {
- _index = index;
- _delegates = delegates;
+ this.index = index;
+ this.delegates = delegates;
}
@Override
public Object get(int i) {
- Pointer ptr = _index[i];
- return _delegates[ptr.listIndex].get(ptr.subIndex);
+ Pointer ptr = index[i];
+ return delegates[ptr.listIndex].get(ptr.subIndex);
}
@Override
public int size() {
- return _index.length;
+ return index.length;
}
public static class Factory implements Serializable {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/tuple/ConsList.java b/storm-client/src/jvm/org/apache/storm/trident/tuple/ConsList.java
index f8acdf0..91647aa 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/tuple/ConsList.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/tuple/ConsList.java
@@ -17,40 +17,40 @@
import java.util.List;
public class ConsList extends AbstractList<Object> {
- List<Object> _elems;
- Object _first;
+ List<Object> elems;
+ Object first;
// for kryo
private ConsList() {
- _elems = new ArrayList<>();
+ elems = new ArrayList<>();
}
public ConsList(Object o, List<Object> elems) {
- _elems = elems;
- _first = o;
+ this.elems = elems;
+ first = o;
}
@Override
public Object get(int i) {
if (i == 0) {
- return _first;
+ return first;
} else {
- return _elems.get(i - 1);
+ return elems.get(i - 1);
}
}
@Override
public int size() {
- return _first == null ? _elems.size() : _elems.size() + 1;
+ return first == null ? elems.size() : elems.size() + 1;
}
// for kryo
@Override
public void add(int index, Object element) {
if (index == 0) {
- _first = element;
+ first = element;
} else {
- _elems.add(index - 1, element);
+ elems.add(index - 1, element);
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/tuple/TridentTupleView.java b/storm-client/src/jvm/org/apache/storm/trident/tuple/TridentTupleView.java
index 893319a..6385477 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/tuple/TridentTupleView.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/tuple/TridentTupleView.java
@@ -24,19 +24,19 @@
import org.apache.storm.tuple.Tuple;
/**
- * Extends AbstractList so that it can be emitted directly as Storm tuples
+ * Extends AbstractList so that it can be emitted directly as Storm tuples.
*/
public class TridentTupleView extends AbstractList<Object> implements TridentTuple {
public static final TridentTupleView EMPTY_TUPLE = new TridentTupleView(null, new ValuePointer[0], new HashMap());
- private final ValuePointer[] _index;
- private final Map<String, ValuePointer> _fieldIndex;
- private final List<List<Object>> _delegates;
+ private final ValuePointer[] index;
+ private final Map<String, ValuePointer> fieldIndex;
+ private final List<List<Object>> delegates;
// index and fieldIndex are precomputed, delegates built up over many operations using persistent data structures
public TridentTupleView(List delegates, ValuePointer[] index, Map<String, ValuePointer> fieldIndex) {
- _delegates = delegates;
- _index = index;
- _fieldIndex = fieldIndex;
+ this.delegates = delegates;
+ this.index = index;
+ this.fieldIndex = fieldIndex;
}
private static List<String> indexToFieldsList(ValuePointer[] index) {
@@ -64,7 +64,7 @@
@Override
public int size() {
- return _index.length;
+ return index.length;
}
@Override
@@ -74,7 +74,7 @@
@Override
public Fields getFields() {
- return new Fields(indexToFieldsList(_index));
+ return new Fields(indexToFieldsList(index));
}
@Override
@@ -94,7 +94,7 @@
@Override
public Object getValue(int i) {
- return getValueByPointer(_index[i]);
+ return getValueByPointer(index[i]);
}
@Override
@@ -144,7 +144,7 @@
@Override
public Object getValueByField(String field) {
- return getValueByPointer(_fieldIndex.get(field));
+ return getValueByPointer(fieldIndex.get(field));
}
@Override
@@ -193,71 +193,71 @@
}
private Object getValueByPointer(ValuePointer ptr) {
- return _delegates.get(ptr.delegateIndex).get(ptr.index);
+ return delegates.get(ptr.delegateIndex).get(ptr.index);
}
public static class ProjectionFactory implements Factory {
- Map<String, ValuePointer> _fieldIndex;
- ValuePointer[] _index;
- Factory _parent;
+ Map<String, ValuePointer> fieldIndex;
+ ValuePointer[] index;
+ Factory parent;
public ProjectionFactory(Factory parent, Fields projectFields) {
- _parent = parent;
+ this.parent = parent;
if (projectFields == null) {
projectFields = new Fields();
}
Map<String, ValuePointer> parentFieldIndex = parent.getFieldIndex();
- _fieldIndex = new HashMap<>();
+ fieldIndex = new HashMap<>();
for (String f : projectFields) {
- _fieldIndex.put(f, parentFieldIndex.get(f));
+ fieldIndex.put(f, parentFieldIndex.get(f));
}
- _index = ValuePointer.buildIndex(projectFields, _fieldIndex);
+ index = ValuePointer.buildIndex(projectFields, fieldIndex);
}
public TridentTuple create(TridentTuple parent) {
- if (_index.length == 0) {
+ if (index.length == 0) {
return EMPTY_TUPLE;
} else {
- return new TridentTupleView(((TridentTupleView) parent)._delegates, _index, _fieldIndex);
+ return new TridentTupleView(((TridentTupleView) parent).delegates, index, fieldIndex);
}
}
@Override
public Map<String, ValuePointer> getFieldIndex() {
- return _fieldIndex;
+ return fieldIndex;
}
@Override
public int numDelegates() {
- return _parent.numDelegates();
+ return parent.numDelegates();
}
@Override
public List<String> getOutputFields() {
- return indexToFieldsList(_index);
+ return indexToFieldsList(index);
}
}
public static class FreshOutputFactory implements Factory {
- Map<String, ValuePointer> _fieldIndex;
- ValuePointer[] _index;
+ Map<String, ValuePointer> fieldIndex;
+ ValuePointer[] index;
public FreshOutputFactory(Fields selfFields) {
- _fieldIndex = new HashMap<>();
+ fieldIndex = new HashMap<>();
for (int i = 0; i < selfFields.size(); i++) {
String field = selfFields.get(i);
- _fieldIndex.put(field, new ValuePointer(0, i, field));
+ fieldIndex.put(field, new ValuePointer(0, i, field));
}
- _index = ValuePointer.buildIndex(selfFields, _fieldIndex);
+ index = ValuePointer.buildIndex(selfFields, fieldIndex);
}
public TridentTuple create(List<Object> selfVals) {
- return new TridentTupleView(Arrays.asList(selfVals), _index, _fieldIndex);
+ return new TridentTupleView(Arrays.asList(selfVals), index, fieldIndex);
}
@Override
public Map<String, ValuePointer> getFieldIndex() {
- return _fieldIndex;
+ return fieldIndex;
}
@Override
@@ -267,22 +267,22 @@
@Override
public List<String> getOutputFields() {
- return indexToFieldsList(_index);
+ return indexToFieldsList(index);
}
}
public static class OperationOutputFactory implements Factory {
- Map<String, ValuePointer> _fieldIndex;
- ValuePointer[] _index;
- Factory _parent;
+ Map<String, ValuePointer> fieldIndex;
+ ValuePointer[] index;
+ Factory parent;
public OperationOutputFactory(Factory parent, Fields selfFields) {
- _parent = parent;
- _fieldIndex = new HashMap<>(parent.getFieldIndex());
+ this.parent = parent;
+ fieldIndex = new HashMap<>(parent.getFieldIndex());
int myIndex = parent.numDelegates();
for (int i = 0; i < selfFields.size(); i++) {
String field = selfFields.get(i);
- _fieldIndex.put(field, new ValuePointer(myIndex, i, field));
+ fieldIndex.put(field, new ValuePointer(myIndex, i, field));
}
List<String> myOrder = new ArrayList<>(parent.getOutputFields());
@@ -296,28 +296,28 @@
myOrder.add(f);
}
- _index = ValuePointer.buildIndex(new Fields(myOrder), _fieldIndex);
+ index = ValuePointer.buildIndex(new Fields(myOrder), fieldIndex);
}
public TridentTuple create(TridentTupleView parent, List<Object> selfVals) {
- List<List<Object>> curr = new ArrayList<>(parent._delegates);
+ List<List<Object>> curr = new ArrayList<>(parent.delegates);
curr.add(selfVals);
- return new TridentTupleView(curr, _index, _fieldIndex);
+ return new TridentTupleView(curr, index, fieldIndex);
}
@Override
public Map<String, ValuePointer> getFieldIndex() {
- return _fieldIndex;
+ return fieldIndex;
}
@Override
public int numDelegates() {
- return _parent.numDelegates() + 1;
+ return parent.numDelegates() + 1;
}
@Override
public List<String> getOutputFields() {
- return indexToFieldsList(_index);
+ return indexToFieldsList(index);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/util/LRUMap.java b/storm-client/src/jvm/org/apache/storm/trident/util/LRUMap.java
index 2d5d0ce..3f26dd3 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/util/LRUMap.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/util/LRUMap.java
@@ -15,16 +15,17 @@
import java.util.LinkedHashMap;
import java.util.Map;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class LRUMap<A, B> extends LinkedHashMap<A, B> {
- private int _maxSize;
+ private int maxSize;
public LRUMap(int maxSize) {
super(maxSize + 1, 1.0f, true);
- _maxSize = maxSize;
+ this.maxSize = maxSize;
}
@Override
protected boolean removeEldestEntry(final Map.Entry<A, B> eldest) {
- return size() > _maxSize;
+ return size() > maxSize;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/util/TridentUtils.java b/storm-client/src/jvm/org/apache/storm/trident/util/TridentUtils.java
index c290375..a52e7e3 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/util/TridentUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/util/TridentUtils.java
@@ -73,7 +73,7 @@
}
/**
- * Assumes edge contains an index
+ * Assumes edge contains an index.
*/
public static <T> List<T> getParents(DirectedGraph g, T n) {
List<IndexedEdge> incoming = new ArrayList(g.incomingEdgesOf(n));
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
index 79d64a1..81451d3 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java
@@ -102,8 +102,6 @@
/**
* Handle expired tuple events which can be removing from cache or store.
- *
- * @param expiredEvents
*/
protected abstract void onTuplesExpired(List<T> expiredEvents);
@@ -132,9 +130,6 @@
/**
* Return {@code TridentTuple}s from given {@code tupleEvents}.
- *
- * @param tupleEvents
- * @return
*/
protected abstract List<TridentTuple> getTridentTuples(List<T> tupleEvents);
@@ -214,10 +209,10 @@
@Override
public String toString() {
- return "TriggerResult{" +
- "id=" + id +
- ", result=" + result +
- '}';
+ return "TriggerResult{"
+ + "id=" + id
+ + ", result=" + result
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/ITridentWindowManager.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/ITridentWindowManager.java
index ab0fb73..cf23afc 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/ITridentWindowManager.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/ITridentWindowManager.java
@@ -24,7 +24,7 @@
/**
* This is invoked from {@code org.apache.storm.trident.planner.TridentProcessor}'s prepare method. So any initialization tasks can be
* done before the topology starts accepting tuples. For ex: initialize window manager with any earlier stored tuples/triggers and start
- * WindowManager
+ * WindowManager.
*/
public void prepare();
@@ -35,17 +35,12 @@
public void shutdown();
/**
- * Add received batch of tuples to cache/store and add them to {@code WindowManager}
- *
- * @param batchId
- * @param tuples
+ * Add received batch of tuples to cache/store and add them to {@code WindowManager}.
*/
public void addTuplesBatch(Object batchId, List<TridentTuple> tuples);
/**
* Returns pending triggers to be emitted.
- *
- * @return
*/
public Queue<StoreBasedTridentWindowManager.TriggerResult> getPendingTriggers();
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/InMemoryWindowsStore.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/InMemoryWindowsStore.java
index e0c6f27..a398fb0 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/InMemoryWindowsStore.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/InMemoryWindowsStore.java
@@ -36,6 +36,7 @@
}
/**
+ * Creates a new in-memory window store.
* @param maxSize maximum size of inmemory store
* @param backingStore backing store containing the entries
*/
@@ -99,14 +100,14 @@
@Override
public void put(String key, Object value) {
- _put(key, value);
+ put0(key, value);
if (backingStore != null) {
backingStore.put(key, value);
}
}
- private void _put(String key, Object value) {
+ private void put0(String key, Object value) {
if (!canAdd()) {
return;
}
@@ -128,7 +129,7 @@
@Override
public void putAll(Collection<Entry> entries) {
for (Entry entry : entries) {
- _put(entry.key, entry.value);
+ put0(entry.key, entry.value);
}
if (backingStore != null) {
backingStore.putAll(entries);
@@ -137,14 +138,14 @@
@Override
public void remove(String key) {
- _remove(key);
+ remove0(key);
if (backingStore != null) {
backingStore.remove(key);
}
}
- private void _remove(String key) {
+ private void remove0(String key) {
Object oldValue = store.remove(key);
if (oldValue != null) {
@@ -165,7 +166,7 @@
@Override
public void removeAll(Collection<String> keys) {
for (String key : keys) {
- _remove(key);
+ remove0(key);
}
if (backingStore != null) {
@@ -184,9 +185,9 @@
@Override
public String toString() {
- return "InMemoryWindowsStore{" +
- " store:size = " + store.size() +
- " backingStore = " + backingStore +
- '}';
+ return "InMemoryWindowsStore{"
+ + " store:size = " + store.size()
+ + " backingStore = " + backingStore
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/TridentBatchTuple.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/TridentBatchTuple.java
index 34efd9d..7b42908 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/TridentBatchTuple.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/TridentBatchTuple.java
@@ -14,9 +14,6 @@
import org.apache.storm.trident.tuple.TridentTuple;
-/**
- *
- */
public class TridentBatchTuple {
final String effectiveBatchId;
final long timeStamp;
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowTridentProcessor.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowTridentProcessor.java
index 077b6a1..c5e7f93 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowTridentProcessor.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowTridentProcessor.java
@@ -107,8 +107,6 @@
throw new RuntimeException("Aggregation related operation can only have one parent");
}
- Long maxTuplesCacheSize = getWindowTuplesCacheSize(topoConf);
-
this.tridentContext = tridentContext;
collector = new FreshCollector(tridentContext);
projection = new TridentTupleView.ProjectionFactory(parents.get(0), inputFields);
@@ -117,10 +115,20 @@
windowTaskId = windowId + WindowsStore.KEY_SEPARATOR + topologyContext.getThisTaskId() + WindowsStore.KEY_SEPARATOR;
windowTriggerInprocessId = getWindowTriggerInprocessIdPrefix(windowTaskId);
- tridentWindowManager = storeTuplesInStore ?
- new StoreBasedTridentWindowManager(windowConfig, windowTaskId, windowStore, aggregator, tridentContext.getDelegateCollector(),
- maxTuplesCacheSize, inputFields)
- : new InMemoryTridentWindowManager(windowConfig, windowTaskId, windowStore, aggregator, tridentContext.getDelegateCollector());
+ Long maxTuplesCacheSize = getWindowTuplesCacheSize(topoConf);
+ tridentWindowManager = storeTuplesInStore
+ ? new StoreBasedTridentWindowManager(windowConfig,
+ windowTaskId,
+ windowStore,
+ aggregator,
+ tridentContext.getDelegateCollector(),
+ maxTuplesCacheSize,
+ inputFields)
+ : new InMemoryTridentWindowManager(windowConfig,
+ windowTaskId,
+ windowStore,
+ aggregator,
+ tridentContext.getDelegateCollector());
tridentWindowManager.prepare();
}
@@ -250,10 +258,10 @@
@Override
public String toString() {
- return "TriggerInfo{" +
- "windowTaskId='" + windowTaskId + '\'' +
- ", triggerId=" + triggerId +
- '}';
+ return "TriggerInfo{"
+ + "windowTaskId='" + windowTaskId + '\''
+ + ", triggerId=" + triggerId
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStateUpdater.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStateUpdater.java
index 58e350e..8d81027 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStateUpdater.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStateUpdater.java
@@ -25,7 +25,7 @@
import org.slf4j.LoggerFactory;
/**
- * {@code StateUpdater<WindowState>} instance which removes successfully emitted triggers from store
+ * {@code StateUpdater<WindowState>} instance which removes successfully emitted triggers from store.
*/
public class WindowsStateUpdater implements StateUpdater<WindowsState> {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStoreFactory.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStoreFactory.java
index 95e9497..b901d51 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStoreFactory.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/WindowsStoreFactory.java
@@ -23,7 +23,7 @@
public interface WindowsStoreFactory extends Serializable {
/**
- * Creates a window store
+ * Creates a window store.
*
* @param topoConf storm topology configuration passed in {@link org.apache.storm.trident.planner.TridentProcessor#prepare(Map,
* TopologyContext, TridentContext)}
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/config/BaseWindowConfig.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/config/BaseWindowConfig.java
index 72794fd..c1b94ad 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/config/BaseWindowConfig.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/config/BaseWindowConfig.java
@@ -12,9 +12,6 @@
package org.apache.storm.trident.windowing.config;
-/**
- *
- */
public abstract class BaseWindowConfig implements WindowConfig {
protected final int windowLength;
protected final int slideLength;
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/config/WindowConfig.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/config/WindowConfig.java
index 1b231c7..7644b8b 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/config/WindowConfig.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/config/WindowConfig.java
@@ -22,22 +22,16 @@
/**
* Returns the length of the window.
- *
- * @return
*/
public int getWindowLength();
/**
* Returns the sliding length of the moving window.
- *
- * @return
*/
public int getSlidingLength();
/**
* Gives the type of windowing. It can be any of {@code Type} values.
- *
- * @return
*/
public <T> WindowStrategy<T> getWindowStrategy();
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/BaseWindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/BaseWindowStrategy.java
index 22bb3b1..0a14924 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/BaseWindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/BaseWindowStrategy.java
@@ -14,9 +14,6 @@
import org.apache.storm.trident.windowing.config.WindowConfig;
-/**
- *
- */
public abstract class BaseWindowStrategy<T> implements WindowStrategy<T> {
protected final WindowConfig windowConfig;
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingCountWindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingCountWindowStrategy.java
index 8c23d15..c4ff190 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingCountWindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingCountWindowStrategy.java
@@ -31,10 +31,6 @@
/**
* Returns a {@code TriggerPolicy} which triggers for every count of given sliding window.
- *
- * @param triggerHandler
- * @param evictionPolicy
- * @return
*/
@Override
public TriggerPolicy<T, ?> getTriggerPolicy(TriggerHandler triggerHandler, EvictionPolicy<T, ?> evictionPolicy) {
@@ -43,8 +39,6 @@
/**
* Returns an {@code EvictionPolicy} instance which evicts elements after a count of given window length.
- *
- * @return
*/
@Override
public EvictionPolicy<T, ?> getEvictionPolicy() {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingDurationWindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingDurationWindowStrategy.java
index c1d57aa..d086fcb 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingDurationWindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/SlidingDurationWindowStrategy.java
@@ -31,10 +31,6 @@
/**
* Returns a {@code TriggerPolicy} which triggers for every configured sliding window duration.
- *
- * @param triggerHandler
- * @param evictionPolicy
- * @return
*/
@Override
public TriggerPolicy<T, ?> getTriggerPolicy(TriggerHandler triggerHandler, EvictionPolicy<T, ?> evictionPolicy) {
@@ -43,8 +39,6 @@
/**
* Returns an {@code EvictionPolicy} instance which evicts elements after window duration is reached.
- *
- * @return
*/
@Override
public EvictionPolicy<T, ?> getEvictionPolicy() {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingCountWindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingCountWindowStrategy.java
index 751a157..85e2f6f 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingCountWindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingCountWindowStrategy.java
@@ -31,10 +31,6 @@
/**
* Returns a {@code TriggerPolicy} which triggers for every count of given sliding window.
- *
- * @param triggerHandler
- * @param evictionPolicy
- * @return
*/
@Override
public TriggerPolicy<T, ?> getTriggerPolicy(TriggerHandler triggerHandler, EvictionPolicy<T, ?> evictionPolicy) {
@@ -43,8 +39,6 @@
/**
* Returns an {@code EvictionPolicy} instance which evicts elements after a count of given window length.
- *
- * @return
*/
@Override
public EvictionPolicy<T, ?> getEvictionPolicy() {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingDurationWindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingDurationWindowStrategy.java
index 79c024a..7e29325 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingDurationWindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/TumblingDurationWindowStrategy.java
@@ -31,10 +31,6 @@
/**
* Returns a {@code TriggerPolicy} which triggers for every given sliding duration.
- *
- * @param triggerHandler
- * @param evictionPolicy
- * @return
*/
@Override
public TriggerPolicy<T, ?> getTriggerPolicy(TriggerHandler triggerHandler, EvictionPolicy<T, ?> evictionPolicy) {
@@ -43,8 +39,6 @@
/**
* Returns an {@code EvictionPolicy} instance which evicts elements after given window duration.
- *
- * @return
*/
@Override
public EvictionPolicy<T, ?> getEvictionPolicy() {
diff --git a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/WindowStrategy.java b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/WindowStrategy.java
index d150f1e..ff814f9 100644
--- a/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/WindowStrategy.java
+++ b/storm-client/src/jvm/org/apache/storm/trident/windowing/strategy/WindowStrategy.java
@@ -23,17 +23,11 @@
/**
* Returns a {@code TriggerPolicy} by creating with {@code triggerHandler} and {@code evictionPolicy} with the given configuration.
- *
- * @param triggerHandler
- * @param evictionPolicy
- * @return
*/
public TriggerPolicy<T, ?> getTriggerPolicy(TriggerHandler triggerHandler, EvictionPolicy<T, ?> evictionPolicy);
/**
* Returns an {@code EvictionPolicy} instance for this strategy with the given configuration.
- *
- * @return
*/
public EvictionPolicy<T, ?> getEvictionPolicy();
}
diff --git a/storm-client/src/jvm/org/apache/storm/tuple/Fields.java b/storm-client/src/jvm/org/apache/storm/tuple/Fields.java
index bdd80b7..c76e365 100644
--- a/storm-client/src/jvm/org/apache/storm/tuple/Fields.java
+++ b/storm-client/src/jvm/org/apache/storm/tuple/Fields.java
@@ -21,33 +21,33 @@
import java.util.Map;
/**
- * Collection of unique named fields using in an ITuple
+ * Collection of unique named fields using in an ITuple.
*/
public class Fields implements Iterable<String>, Serializable {
private static final long serialVersionUID = -3377931843059975424L;
- private List<String> _fields;
- private Map<String, Integer> _index = new HashMap<>();
+ private List<String> fields;
+ private Map<String, Integer> index = new HashMap<>();
public Fields(String... fields) {
this(Arrays.asList(fields));
}
public Fields(List<String> fields) {
- _fields = new ArrayList<>(fields.size());
+ this.fields = new ArrayList<>(fields.size());
for (String field : fields) {
- if (_fields.contains(field)) {
+ if (this.fields.contains(field)) {
throw new IllegalArgumentException(
String.format("duplicate field '%s'", field)
);
}
- _fields.add(field);
+ this.fields.add(field);
}
index();
}
/**
* Select values out of tuple given a Fields selector Note that this function can throw a NullPointerException if the fields in selector
- * are not found in the _index
+ * are not found in the index.
*
* @param selector Fields to select
* @param tuple tuple to select from
@@ -61,14 +61,14 @@
}
public List<String> toList() {
- return new ArrayList<>(_fields);
+ return new ArrayList<>(fields);
}
/**
* Returns the number of fields in this collection.
*/
public int size() {
- return _fields.size();
+ return fields.size();
}
/**
@@ -78,12 +78,12 @@
* @throws IndexOutOfBoundsException - if the index is out of range (index < 0 || index >= size())
*/
public String get(int index) {
- return _fields.get(index);
+ return fields.get(index);
}
@Override
public Iterator<String> iterator() {
- return _fields.iterator();
+ return fields.iterator();
}
/**
@@ -93,7 +93,7 @@
* @throws IllegalArgumentException - if field does not exist
*/
public int fieldIndex(String field) {
- Integer ret = _index.get(field);
+ Integer ret = index.get(field);
if (ret == null) {
throw new IllegalArgumentException(field + " does not exist");
}
@@ -101,21 +101,22 @@
}
/**
+ * Check contains.
* @return true if this contains the specified name of the field.
*/
public boolean contains(String field) {
- return _index.containsKey(field);
+ return index.containsKey(field);
}
private void index() {
- for (int i = 0; i < _fields.size(); i++) {
- _index.put(_fields.get(i), i);
+ for (int i = 0; i < fields.size(); i++) {
+ index.put(fields.get(i), i);
}
}
@Override
public String toString() {
- return _fields.toString();
+ return fields.toString();
}
@Override
@@ -125,13 +126,13 @@
}
if (other instanceof Fields) {
Fields of = (Fields) other;
- return _fields.equals(of._fields);
+ return fields.equals(of.fields);
}
return false;
}
@Override
public int hashCode() {
- return _fields.hashCode();
+ return fields.hashCode();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/tuple/MessageId.java b/storm-client/src/jvm/org/apache/storm/tuple/MessageId.java
index 5a7b5ea..cb10dd1 100644
--- a/storm-client/src/jvm/org/apache/storm/tuple/MessageId.java
+++ b/storm-client/src/jvm/org/apache/storm/tuple/MessageId.java
@@ -23,12 +23,12 @@
import java.util.Set;
public class MessageId {
- final static MessageId unanchoredMsgId = makeId(Collections.emptyMap());
+ static final MessageId unanchoredMsgId = makeId(Collections.emptyMap());
- private Map<Long, Long> _anchorsToIds;
+ private Map<Long, Long> anchorsToIds;
protected MessageId(Map<Long, Long> anchorsToIds) {
- _anchorsToIds = anchorsToIds;
+ this.anchorsToIds = anchorsToIds;
}
public static long generateId(Random rand) {
@@ -59,31 +59,31 @@
}
public Map<Long, Long> getAnchorsToIds() {
- return _anchorsToIds;
+ return anchorsToIds;
}
public Set<Long> getAnchors() {
- return _anchorsToIds.keySet();
+ return anchorsToIds.keySet();
}
@Override
public int hashCode() {
- return _anchorsToIds.hashCode();
+ return anchorsToIds.hashCode();
}
@Override
public boolean equals(Object other) {
- return other instanceof MessageId && _anchorsToIds.equals(((MessageId) other)._anchorsToIds);
+ return other instanceof MessageId && anchorsToIds.equals(((MessageId) other).anchorsToIds);
}
@Override
public String toString() {
- return _anchorsToIds.toString();
+ return anchorsToIds.toString();
}
public void serialize(Output out) throws IOException {
- out.writeInt(_anchorsToIds.size(), true);
- for (Entry<Long, Long> anchorToId : _anchorsToIds.entrySet()) {
+ out.writeInt(anchorsToIds.size(), true);
+ for (Entry<Long, Long> anchorToId : anchorsToIds.entrySet()) {
out.writeLong(anchorToId.getKey());
out.writeLong(anchorToId.getValue());
}
diff --git a/storm-client/src/jvm/org/apache/storm/tuple/Tuple.java b/storm-client/src/jvm/org/apache/storm/tuple/Tuple.java
index ec91d77..01d6c88 100644
--- a/storm-client/src/jvm/org/apache/storm/tuple/Tuple.java
+++ b/storm-client/src/jvm/org/apache/storm/tuple/Tuple.java
@@ -20,7 +20,7 @@
* dynamically typed -- the types of the fields do not need to be declared. Tuples have helper methods like getInteger and getString to get
* field values without having to cast the result.
*
- * Storm needs to know how to serialize all the values in a tuple. By default, Storm knows how to serialize the primitive types, strings,
+ * <p>Storm needs to know how to serialize all the values in a tuple. By default, Storm knows how to serialize the primitive types, strings,
* and byte arrays. If you want to use another type, you'll need to implement and register a serializer for that type.
*
* @see <a href="http://storm.apache.org/documentation/Serialization.html">Serialization</a>
@@ -53,7 +53,7 @@
public MessageId getMessageId();
/**
- * Gets the topology context associated with the tuple
+ * Gets the topology context associated with the tuple.
*/
public GeneralTopologyContext getContext();
}
diff --git a/storm-client/src/jvm/org/apache/storm/tuple/TupleImpl.java b/storm-client/src/jvm/org/apache/storm/tuple/TupleImpl.java
index 27fa958..e0a2827 100644
--- a/storm-client/src/jvm/org/apache/storm/tuple/TupleImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/tuple/TupleImpl.java
@@ -24,9 +24,9 @@
private String streamId;
private GeneralTopologyContext context;
private MessageId id;
- private Long _processSampleStartTime;
- private Long _executeSampleStartTime;
- private long _outAckVal = 0;
+ private Long processSampleStartTime;
+ private Long executeSampleStartTime;
+ private long outAckVal = 0;
public TupleImpl(Tuple t) {
this.values = t.getValues();
@@ -37,9 +37,9 @@
this.srcComponent = t.getSourceComponent();
try {
TupleImpl ti = (TupleImpl) t;
- this._processSampleStartTime = ti._processSampleStartTime;
- this._executeSampleStartTime = ti._executeSampleStartTime;
- this._outAckVal = ti._outAckVal;
+ this.processSampleStartTime = ti.processSampleStartTime;
+ this.executeSampleStartTime = ti.executeSampleStartTime;
+ this.outAckVal = ti.outAckVal;
} catch (ClassCastException e) {
// ignore ... if t is not a TupleImpl type .. faster than checking and then casting
}
@@ -68,32 +68,29 @@
}
public Long getProcessSampleStartTime() {
- return _processSampleStartTime;
+ return processSampleStartTime;
}
public void setProcessSampleStartTime(long ms) {
- _processSampleStartTime = ms;
+ processSampleStartTime = ms;
}
public Long getExecuteSampleStartTime() {
- return _executeSampleStartTime;
+ return executeSampleStartTime;
}
public void setExecuteSampleStartTime(long ms) {
- _executeSampleStartTime = ms;
+ executeSampleStartTime = ms;
}
public void updateAckVal(long val) {
- _outAckVal = _outAckVal ^ val;
+ outAckVal = outAckVal ^ val;
}
public long getAckVal() {
- return _outAckVal;
+ return outAckVal;
}
- /**
- * Tuple APIs
- */
@Override
public int size() {
return values.size();
@@ -256,9 +253,11 @@
@Override
public String toString() {
- return "source: " + getSourceComponent() + ":" + taskId + ", stream: " + streamId + ", id: " + id.toString() + ", " +
- values.toString() + " PROC_START_TIME(sampled): " + _processSampleStartTime + " EXEC_START_TIME(sampled): " +
- _executeSampleStartTime;
+ return "source: " + getSourceComponent() + ":" + taskId
+ + ", stream: " + streamId
+ + ", id: " + id.toString()
+ + ", " + values.toString() + " PROC_START_TIME(sampled): "
+ + processSampleStartTime + " EXEC_START_TIME(sampled): " + executeSampleStartTime;
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/utils/CRC32OutputStream.java b/storm-client/src/jvm/org/apache/storm/utils/CRC32OutputStream.java
index 86940bd..7401ac5 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/CRC32OutputStream.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/CRC32OutputStream.java
@@ -16,6 +16,7 @@
import java.io.OutputStream;
import java.util.zip.CRC32;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class CRC32OutputStream extends OutputStream {
private CRC32 hasher;
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
index 424407c..745be60 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -27,10 +27,10 @@
import java.util.function.BooleanSupplier;
import java.util.stream.Collectors;
-import org.apache.storm.shade.com.google.common.collect.Maps;
import org.apache.storm.Config;
import org.apache.storm.daemon.supervisor.AdvancedFSOps;
import org.apache.storm.generated.StormTopology;
+import org.apache.storm.shade.com.google.common.collect.Maps;
import org.apache.storm.shade.org.apache.commons.io.FileUtils;
import org.apache.storm.validation.ConfigValidation;
import org.apache.storm.validation.ConfigValidationAnnotations;
@@ -38,7 +38,7 @@
public class ConfigUtils {
public static final String FILE_SEPARATOR = File.separator;
public static final String STORM_HOME = "storm.home";
- public final static String RESOURCES_SUBDIR = "resources";
+ public static final String RESOURCES_SUBDIR = "resources";
private static final Set<String> passwordConfigKeys = new HashSet<>();
@@ -63,7 +63,7 @@
// A singleton instance allows us to mock delegated static methods in our
// tests by subclassing.
- private static ConfigUtils _instance = new ConfigUtils();
+ private static ConfigUtils instance = new ConfigUtils();
/**
* Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that
@@ -73,18 +73,17 @@
* @return the previously set instance
*/
public static ConfigUtils setInstance(ConfigUtils u) {
- ConfigUtils oldInstance = _instance;
- _instance = u;
+ ConfigUtils oldInstance = instance;
+ instance = u;
return oldInstance;
}
public static Map<String, Object> maskPasswords(final Map<String, Object> conf) {
- Maps.EntryTransformer<String, Object, Object> maskPasswords =
- new Maps.EntryTransformer<String, Object, Object>() {
- public Object transformEntry(String key, Object value) {
- return passwordConfigKeys.contains(key) ? "*****" : value;
- }
- };
+ Maps.EntryTransformer<String, Object, Object> maskPasswords = new Maps.EntryTransformer<String, Object, Object>() {
+ public Object transformEntry(String key, Object value) {
+ return passwordConfigKeys.contains(key) ? "*****" : value;
+ }
+ };
return Maps.transformEntries(conf, maskPasswords);
}
@@ -132,7 +131,7 @@
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static String workerArtifactsRoot(Map<String, Object> conf) {
- return _instance.workerArtifactsRootImpl(conf);
+ return instance.workerArtifactsRootImpl(conf);
}
public static String workerArtifactsRoot(Map<String, Object> conf, String id) {
@@ -164,7 +163,7 @@
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static Map<String, Object> readStormConfig() {
- return _instance.readStormConfigImpl();
+ return instance.readStormConfigImpl();
}
public static int samplingRate(Map<String, Object> conf) {
@@ -199,7 +198,7 @@
}
public static StormTopology readSupervisorTopology(Map<String, Object> conf, String stormId, AdvancedFSOps ops) throws IOException {
- return _instance.readSupervisorTopologyImpl(conf, stormId, ops);
+ return instance.readSupervisorTopologyImpl(conf, stormId, ops);
}
public static String supervisorStormCodePath(String stormRoot) {
@@ -217,12 +216,12 @@
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static String supervisorStormDistRoot(Map<String, Object> conf) throws IOException {
- return ConfigUtils._instance.supervisorStormDistRootImpl(conf);
+ return ConfigUtils.instance.supervisorStormDistRootImpl(conf);
}
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static String supervisorStormDistRoot(Map<String, Object> conf, String stormId) throws IOException {
- return _instance.supervisorStormDistRootImpl(conf, stormId);
+ return instance.supervisorStormDistRootImpl(conf, stormId);
}
public static String supervisorStormJarPath(String stormRoot) {
@@ -271,7 +270,7 @@
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static String workerRoot(Map<String, Object> conf) {
- return _instance.workerRootImpl(conf);
+ return instance.workerRootImpl(conf);
}
public static String workerRoot(Map<String, Object> conf, String id) {
@@ -286,13 +285,17 @@
return (workerPidsRoot(conf, id) + FILE_SEPARATOR + pid);
}
+ public static String workerPidPath(Map<String, Object> conf, String id, long pid) {
+ return workerPidPath(conf, id, String.valueOf(pid));
+ }
+
public static String workerArtifactsPidPath(Map<String, Object> conf, String id, Integer port) {
return (workerArtifactsRoot(conf, id, port) + FILE_SEPARATOR + "worker.pid");
}
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static Map<String, Object> readSupervisorStormConf(Map<String, Object> conf, String stormId) throws IOException {
- return _instance.readSupervisorStormConfImpl(conf, stormId);
+ return instance.readSupervisorStormConfImpl(conf, stormId);
}
public static Map<String, Object> readSupervisorStormConfGivenPath(Map<String, Object> conf, String topoConfPath) throws IOException {
@@ -330,17 +333,17 @@
if (key == null) {
return null;
}
- final String STORM_JAR_SUFFIX = "-stormjar.jar";
- final String STORM_CODE_SUFFIX = "-stormcode.ser";
- final String STORM_CONF_SUFFIX = "-stormconf.ser";
+ final String stormJarSuffix = "-stormjar.jar";
+ final String stormCodeSuffix = "-stormcode.ser";
+ final String stormConfSuffix = "-stormconf.ser";
String ret = null;
- if (key.endsWith(STORM_JAR_SUFFIX)) {
- ret = key.substring(0, key.length() - STORM_JAR_SUFFIX.length());
- } else if (key.endsWith(STORM_CODE_SUFFIX)) {
- ret = key.substring(0, key.length() - STORM_CODE_SUFFIX.length());
- } else if (key.endsWith(STORM_CONF_SUFFIX)) {
- ret = key.substring(0, key.length() - STORM_CONF_SUFFIX.length());
+ if (key.endsWith(stormJarSuffix)) {
+ ret = key.substring(0, key.length() - stormJarSuffix.length());
+ } else if (key.endsWith(stormCodeSuffix)) {
+ ret = key.substring(0, key.length() - stormCodeSuffix.length());
+ } else if (key.endsWith(stormConfSuffix)) {
+ ret = key.substring(0, key.length() - stormConfSuffix.length());
}
return ret;
}
@@ -370,7 +373,7 @@
// we use this "weird" wrapper pattern temporarily for mocking in clojure test
public static String supervisorLocalDir(Map<String, Object> conf) throws IOException {
- return _instance.supervisorLocalDirImpl(conf);
+ return instance.supervisorLocalDirImpl(conf);
}
public static String workerTmpRoot(Map<String, Object> conf, String id) {
@@ -385,10 +388,6 @@
return (workerUserRoot(conf) + FILE_SEPARATOR + workerId);
}
- public static String workerPidPath(Map<String, Object> conf, String id, long pid) {
- return workerPidPath(conf, id, String.valueOf(pid));
- }
-
public static File getWorkerDirFromRoot(String logRoot, String id, Integer port) {
return new File((logRoot + FILE_SEPARATOR + id + FILE_SEPARATOR + port));
}
@@ -451,6 +450,10 @@
return supervisorStormDistRoot(conf) + FILE_SEPARATOR + Utils.urlEncodeUtf8(stormId);
}
+ public String supervisorStormDistRootImpl(Map<String, Object> conf) throws IOException {
+ return stormDistPath(supervisorLocalDir(conf));
+ }
+
public String workerRootImpl(Map<String, Object> conf) {
return (absoluteStormLocalDir(conf) + FILE_SEPARATOR + "workers");
}
@@ -461,10 +464,6 @@
return readSupervisorStormConfGivenPath(conf, confPath);
}
- public String supervisorStormDistRootImpl(Map<String, Object> conf) throws IOException {
- return stormDistPath(supervisorLocalDir(conf));
- }
-
public String supervisorLocalDirImpl(Map<String, Object> conf) throws IOException {
String ret = ConfigUtils.absoluteStormLocalDir(conf) + FILE_SEPARATOR + "supervisor";
FileUtils.forceMkdir(new File(ret));
diff --git a/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java b/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java
index 9cfb9e1..0f23d22 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/CuratorUtils.java
@@ -79,31 +79,32 @@
if (!exhibitorServers.isEmpty()) {
// use exhibitor servers
builder.ensembleProvider(new ExhibitorEnsembleProvider(
- new Exhibitors(exhibitorServers, ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_PORT)),
- new Exhibitors.BackupConnectionStringProvider() {
- @Override
- public String getBackupConnectionString() throws Exception {
- // use zk servers as backup if they exist
- return zkStr;
- }
- }),
+ new Exhibitors(exhibitorServers,
+ ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_PORT)),
+ new Exhibitors.BackupConnectionStringProvider() {
+ @Override
+ public String getBackupConnectionString() throws Exception {
+ // use zk servers as backup if they exist
+ return zkStr;
+ }
+ }),
new DefaultExhibitorRestClient(),
ObjectReader.getString(conf.get(Config.STORM_EXHIBITOR_URIPATH)),
ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_POLL)),
new StormBoundedExponentialBackoffRetry(
- ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_INTERVAL)),
- ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_INTERVAL_CEILING)),
- ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_TIMES)))));
+ ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_INTERVAL)),
+ ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_INTERVAL_CEILING)),
+ ObjectReader.getInt(conf.get(Config.STORM_EXHIBITOR_RETRY_TIMES)))));
} else {
builder.connectString(zkStr);
}
builder
- .connectionTimeoutMs(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
- .sessionTimeoutMs(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
- .retryPolicy(new StormBoundedExponentialBackoffRetry(
- ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)),
- ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)),
- ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES))));
+ .connectionTimeoutMs(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
+ .sessionTimeoutMs(ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
+ .retryPolicy(new StormBoundedExponentialBackoffRetry(
+ ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)),
+ ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)),
+ ObjectReader.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES))));
if (auth != null && auth.scheme != null && auth.payload != null) {
builder.authorization(auth.scheme, auth.payload);
diff --git a/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java b/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
index 85a4c4b..6cf4bdf 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
@@ -29,9 +29,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class DRPCClient extends ThriftClient implements DistributedRPC.Iface {
private static final Logger LOG = LoggerFactory.getLogger(DRPCClient.class);
- private static volatile ILocalDRPC _localOverrideClient = null;
+ private static volatile ILocalDRPC localOverrideClient = null;
private DistributedRPC.Iface client;
private String host;
private int port;
@@ -46,38 +47,40 @@
public DRPCClient(Map<String, Object> conf, String host, int port) throws TTransportException {
this(conf, host, port, null);
- _retryForever = true;
+ retryForever = true;
}
public DRPCClient(Map<String, Object> conf, String host, int port, Integer timeout) throws TTransportException {
- super(conf, _localOverrideClient != null ? ThriftConnectionType.LOCAL_FAKE : ThriftConnectionType.DRPC,
+ super(conf, localOverrideClient != null ? ThriftConnectionType.LOCAL_FAKE : ThriftConnectionType.DRPC,
host, port, timeout, null);
this.host = host;
this.port = port;
- if (_localOverrideClient != null) {
- this.client = _localOverrideClient;
+ if (localOverrideClient != null) {
+ this.client = localOverrideClient;
} else {
- this.client = new DistributedRPC.Client(_protocol);
+ this.client = new DistributedRPC.Client(protocol);
}
- _retryForever = true;
+ retryForever = true;
}
/**
- * @return true of new clients will be overridden to connect to a local cluster and not the configured remote cluster.
+ * Check local override.
+ * @return true of new clients will be overridden to connect to a local cluster and not the configured remote cluster
*/
public static boolean isLocalOverride() {
- return _localOverrideClient != null;
+ return localOverrideClient != null;
}
/**
+ * Get override service ID.
* @return the service ID of the local override DRPC instance
*/
public static String getOverrideServiceId() {
- return _localOverrideClient.getServiceId();
+ return localOverrideClient.getServiceId();
}
public static DRPCClient getConfiguredClient(Map<String, Object> conf) throws TTransportException {
- DistributedRPC.Iface override = _localOverrideClient;
+ DistributedRPC.Iface override = localOverrideClient;
if (override != null) {
return new DRPCClient(override);
}
@@ -134,12 +137,12 @@
public static class LocalOverride implements AutoCloseable {
public LocalOverride(ILocalDRPC client) {
- _localOverrideClient = client;
+ localOverrideClient = client;
}
@Override
public void close() throws Exception {
- _localOverrideClient = null;
+ localOverrideClient = null;
}
}
}
\ No newline at end of file
diff --git a/storm-client/src/jvm/org/apache/storm/utils/InprocMessaging.java b/storm-client/src/jvm/org/apache/storm/utils/InprocMessaging.java
index 2accc1d..9ecae46 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/InprocMessaging.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/InprocMessaging.java
@@ -27,7 +27,7 @@
private static ConcurrentMap<Integer, AtomicBoolean> _hasReader = new ConcurrentHashMap<>();
private static int port = 1;
- public synchronized static int acquireNewPort() {
+ public static synchronized int acquireNewPort() {
int ret = port;
port++;
return ret;
@@ -84,7 +84,7 @@
getHasReader(port).set(true);
}
- private synchronized static LinkedBlockingQueue<Object> getQueue(int port) {
+ private static synchronized LinkedBlockingQueue<Object> getQueue(int port) {
if (!_queues.containsKey(port)) {
_queues.put(port, new LinkedBlockingQueue<Object>());
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/JCQueue.java b/storm-client/src/jvm/org/apache/storm/utils/JCQueue.java
index bbeff7a..063447a 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/JCQueue.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/JCQueue.java
@@ -37,6 +37,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class JCQueue implements IStatefulObject, Closeable {
private static final Logger LOG = LoggerFactory.getLogger(JCQueue.class);
private static final String PREFIX = "jc-";
@@ -95,7 +96,7 @@
}
/**
- * Non blocking. Returns immediately if Q is empty. Returns number of elements consumed from Q
+ * Non blocking. Returns immediately if Q is empty. Returns number of elements consumed from Q.
*/
public int consume(JCQueue.Consumer consumer) {
return consume(consumer, continueRunning);
@@ -103,7 +104,7 @@
/**
* Non blocking. Returns immediately if Q is empty. Runs till Q is empty OR exitCond.keepRunning() return false. Returns number of
- * elements consumed from Q
+ * elements consumed from Q.
*/
public int consume(JCQueue.Consumer consumer, ExitCondition exitCond) {
try {
@@ -118,10 +119,7 @@
}
/**
- * Non blocking. Returns immediately if Q is empty. Returns number of elements consumed from Q
- *
- * @param consumer
- * @param exitCond
+ * Non blocking. Returns immediately if Q is empty. Returns number of elements consumed from Q.
*/
private int consumeImpl(Consumer consumer, ExitCondition exitCond) throws InterruptedException {
int drainCount = 0;
@@ -161,11 +159,11 @@
private int tryPublishInternal(ArrayList<Object> objs) {
MessagePassingQueue.Supplier<Object> supplier =
new MessagePassingQueue.Supplier<Object>() {
- int i = 0;
+ int counter = 0;
@Override
public Object get() {
- return objs.get(i++);
+ return objs.get(counter++);
}
};
int count = recvQueue.fill(supplier, objs.size());
@@ -289,10 +287,10 @@
/* Thread safe. Same instance can be used across multiple threads */
private static class DirectInserter implements Inserter {
- private JCQueue q;
+ private JCQueue queue;
- public DirectInserter(JCQueue q) {
- this.q = q;
+ public DirectInserter(JCQueue queue) {
+ this.queue = queue;
}
/**
@@ -300,19 +298,19 @@
*/
@Override
public void publish(Object obj) throws InterruptedException {
- boolean inserted = q.tryPublishInternal(obj);
+ boolean inserted = queue.tryPublishInternal(obj);
int idleCount = 0;
while (!inserted) {
- q.metrics.notifyInsertFailure();
+ queue.metrics.notifyInsertFailure();
if (idleCount == 0) { // check avoids multiple log msgs when in a idle loop
- LOG.debug("Experiencing Back Pressure on recvQueue: '{}'. Entering BackPressure Wait", q.getName());
+ LOG.debug("Experiencing Back Pressure on recvQueue: '{}'. Entering BackPressure Wait", queue.getName());
}
- idleCount = q.backPressureWaitStrategy.idle(idleCount);
+ idleCount = queue.backPressureWaitStrategy.idle(idleCount);
if (Thread.interrupted()) {
throw new InterruptedException();
}
- inserted = q.tryPublishInternal(obj);
+ inserted = queue.tryPublishInternal(obj);
}
}
@@ -322,9 +320,9 @@
*/
@Override
public boolean tryPublish(Object obj) {
- boolean inserted = q.tryPublishInternal(obj);
+ boolean inserted = queue.tryPublishInternal(obj);
if (!inserted) {
- q.metrics.notifyInsertFailure();
+ queue.metrics.notifyInsertFailure();
return false;
}
return true;
@@ -343,11 +341,11 @@
/* Not thread safe. Have one instance per producer thread or synchronize externally */
private static class BatchInserter implements Inserter {
private final int batchSz;
- private JCQueue q;
+ private JCQueue queue;
private ArrayList<Object> currentBatch;
- public BatchInserter(JCQueue q, int batchSz) {
- this.q = q;
+ public BatchInserter(JCQueue queue, int batchSz) {
+ this.queue = queue;
this.batchSz = batchSz;
this.currentBatch = new ArrayList<>(batchSz + 1);
}
@@ -386,18 +384,18 @@
if (currentBatch.isEmpty()) {
return;
}
- int publishCount = q.tryPublishInternal(currentBatch);
+ int publishCount = queue.tryPublishInternal(currentBatch);
int retryCount = 0;
while (publishCount == 0) { // retry till at least 1 element is drained
- q.metrics.notifyInsertFailure();
+ queue.metrics.notifyInsertFailure();
if (retryCount == 0) { // check avoids multiple log msgs when in a idle loop
- LOG.debug("Experiencing Back Pressure when flushing batch to Q: {}. Entering BackPressure Wait.", q.getName());
+ LOG.debug("Experiencing Back Pressure when flushing batch to Q: {}. Entering BackPressure Wait.", queue.getName());
}
- retryCount = q.backPressureWaitStrategy.idle(retryCount);
+ retryCount = queue.backPressureWaitStrategy.idle(retryCount);
if (Thread.interrupted()) {
throw new InterruptedException();
}
- publishCount = q.tryPublishInternal(currentBatch);
+ publishCount = queue.tryPublishInternal(currentBatch);
}
currentBatch.subList(0, publishCount).clear();
}
@@ -411,9 +409,9 @@
if (currentBatch.isEmpty()) {
return true;
}
- int publishCount = q.tryPublishInternal(currentBatch);
+ int publishCount = queue.tryPublishInternal(currentBatch);
if (publishCount == 0) {
- q.metrics.notifyInsertFailure();
+ queue.metrics.notifyInsertFailure();
return false;
} else {
currentBatch.subList(0, publishCount).clear();
diff --git a/storm-client/src/jvm/org/apache/storm/utils/KeyedRoundRobinQueue.java b/storm-client/src/jvm/org/apache/storm/utils/KeyedRoundRobinQueue.java
index 40d16d5..9836a92 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/KeyedRoundRobinQueue.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/KeyedRoundRobinQueue.java
@@ -21,41 +21,41 @@
import java.util.concurrent.Semaphore;
public class KeyedRoundRobinQueue<V> {
- private final Object _lock = new Object();
- private Semaphore _size = new Semaphore(0);
- private Map<Object, Queue<V>> _queues = new HashMap<>();
- private List<Object> _keyOrder = new ArrayList<>();
- private int _currIndex = 0;
+ private final Object lock = new Object();
+ private Semaphore size = new Semaphore(0);
+ private Map<Object, Queue<V>> queues = new HashMap<>();
+ private List<Object> keyOrder = new ArrayList<>();
+ private int currIndex = 0;
public void add(Object key, V val) {
- synchronized (_lock) {
- Queue<V> queue = _queues.get(key);
+ synchronized (lock) {
+ Queue<V> queue = queues.get(key);
if (queue == null) {
queue = new LinkedList<>();
- _queues.put(key, queue);
- _keyOrder.add(key);
+ queues.put(key, queue);
+ keyOrder.add(key);
}
queue.add(val);
}
- _size.release();
+ size.release();
}
public V take() throws InterruptedException {
- _size.acquire();
- synchronized (_lock) {
- Object key = _keyOrder.get(_currIndex);
- Queue<V> queue = _queues.get(key);
+ size.acquire();
+ synchronized (lock) {
+ Object key = keyOrder.get(currIndex);
+ Queue<V> queue = queues.get(key);
V ret = queue.remove();
if (queue.isEmpty()) {
- _keyOrder.remove(_currIndex);
- _queues.remove(key);
- if (_keyOrder.size() == 0) {
- _currIndex = 0;
+ keyOrder.remove(currIndex);
+ queues.remove(key);
+ if (keyOrder.size() == 0) {
+ currIndex = 0;
} else {
- _currIndex = _currIndex % _keyOrder.size();
+ currIndex = currIndex % keyOrder.size();
}
} else {
- _currIndex = (_currIndex + 1) % _keyOrder.size();
+ currIndex = (currIndex + 1) % keyOrder.size();
}
return ret;
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ListDelegate.java b/storm-client/src/jvm/org/apache/storm/utils/ListDelegate.java
index c13bdc1..dea4559 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ListDelegate.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ListDelegate.java
@@ -19,133 +19,133 @@
import java.util.ListIterator;
public class ListDelegate implements List<Object> {
- private List<Object> _delegate;
+ private List<Object> delegate;
public ListDelegate() {
- _delegate = new ArrayList<>();
+ delegate = new ArrayList<>();
}
public List<Object> getDelegate() {
- return _delegate;
+ return delegate;
}
public void setDelegate(List<Object> delegate) {
- _delegate = delegate;
+ this.delegate = delegate;
}
@Override
public int size() {
- return _delegate.size();
+ return delegate.size();
}
@Override
public boolean isEmpty() {
- return _delegate.isEmpty();
+ return delegate.isEmpty();
}
@Override
public boolean contains(Object o) {
- return _delegate.contains(o);
+ return delegate.contains(o);
}
@Override
public Iterator<Object> iterator() {
- return _delegate.iterator();
+ return delegate.iterator();
}
@Override
public Object[] toArray() {
- return _delegate.toArray();
+ return delegate.toArray();
}
@Override
public <T> T[] toArray(T[] ts) {
- return _delegate.toArray(ts);
+ return delegate.toArray(ts);
}
@Override
public boolean add(Object e) {
- return _delegate.add(e);
- }
-
- @Override
- public boolean remove(Object o) {
- return _delegate.remove(o);
- }
-
- @Override
- public boolean containsAll(Collection<?> clctn) {
- return _delegate.containsAll(clctn);
- }
-
- @Override
- public boolean addAll(Collection<?> clctn) {
- return _delegate.addAll(clctn);
- }
-
- @Override
- public boolean addAll(int i, Collection<?> clctn) {
- return _delegate.addAll(i, clctn);
- }
-
- @Override
- public boolean removeAll(Collection<?> clctn) {
- return _delegate.removeAll(clctn);
- }
-
- @Override
- public boolean retainAll(Collection<?> clctn) {
- return _delegate.retainAll(clctn);
- }
-
- @Override
- public void clear() {
- _delegate.clear();
- }
-
- @Override
- public Object get(int i) {
- return _delegate.get(i);
- }
-
- @Override
- public Object set(int i, Object e) {
- return _delegate.set(i, e);
+ return delegate.add(e);
}
@Override
public void add(int i, Object e) {
- _delegate.add(i, e);
+ delegate.add(i, e);
+ }
+
+ @Override
+ public boolean remove(Object o) {
+ return delegate.remove(o);
}
@Override
public Object remove(int i) {
- return _delegate.remove(i);
+ return delegate.remove(i);
+ }
+
+ @Override
+ public boolean containsAll(Collection<?> clctn) {
+ return delegate.containsAll(clctn);
+ }
+
+ @Override
+ public boolean addAll(Collection<?> clctn) {
+ return delegate.addAll(clctn);
+ }
+
+ @Override
+ public boolean addAll(int i, Collection<?> clctn) {
+ return delegate.addAll(i, clctn);
+ }
+
+ @Override
+ public boolean removeAll(Collection<?> clctn) {
+ return delegate.removeAll(clctn);
+ }
+
+ @Override
+ public boolean retainAll(Collection<?> clctn) {
+ return delegate.retainAll(clctn);
+ }
+
+ @Override
+ public void clear() {
+ delegate.clear();
+ }
+
+ @Override
+ public Object get(int i) {
+ return delegate.get(i);
+ }
+
+ @Override
+ public Object set(int i, Object e) {
+ return delegate.set(i, e);
}
@Override
public int indexOf(Object o) {
- return _delegate.indexOf(o);
+ return delegate.indexOf(o);
}
@Override
public int lastIndexOf(Object o) {
- return _delegate.lastIndexOf(o);
+ return delegate.lastIndexOf(o);
}
@Override
public ListIterator<Object> listIterator() {
- return _delegate.listIterator();
+ return delegate.listIterator();
}
@Override
public ListIterator<Object> listIterator(int i) {
- return _delegate.listIterator(i);
+ return delegate.listIterator(i);
}
@Override
public List<Object> subList(int i, int i1) {
- return _delegate.subList(i, i1);
+ return delegate.subList(i, i1);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/LocalState.java b/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
index b2ca316..6e1b44f 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
@@ -45,11 +45,11 @@
public static final String LS_LOCAL_ASSIGNMENTS = "local-assignments";
public static final String LS_APPROVED_WORKERS = "approved-workers";
public static final String LS_TOPO_HISTORY = "topo-hist";
- private VersionedStore _vs;
+ private VersionedStore versionedStore;
public LocalState(String backingDir, boolean createBackingDir) throws IOException {
LOG.debug("New Local State for {}", backingDir);
- _vs = new VersionedStore(backingDir, createBackingDir);
+ versionedStore = new VersionedStore(backingDir, createBackingDir);
}
public synchronized Map<String, TBase> snapshot() {
@@ -94,7 +94,7 @@
private Map<String, ThriftSerializedObject> partialDeserializeLatestVersion(TDeserializer td) {
try {
- String latestPath = _vs.mostRecentVersionPath();
+ String latestPath = versionedStore.mostRecentVersionPath();
Map<String, ThriftSerializedObject> result = new HashMap<>();
if (latestPath != null) {
byte[] serialized = FileUtils.readFileToByteArray(new File(latestPath));
@@ -162,7 +162,7 @@
}
public synchronized void cleanup(int keepVersions) throws IOException {
- _vs.cleanup(keepVersions);
+ versionedStore.cleanup(keepVersions);
}
public List<LSTopoHistory> getTopoHistoryList() {
@@ -175,8 +175,6 @@
/**
* Remove topologies from local state which are older than cutOffAge.
- *
- * @param cutOffAge
*/
public void filterOldTopologies(long cutOffAge) {
LSTopoHistoryList lsTopoHistoryListWrapper = (LSTopoHistoryList) get(LS_TOPO_HISTORY);
@@ -252,7 +250,7 @@
}
byte[] toWrite = ser.serialize(new LocalStateData(serialized));
- String newPath = _vs.createVersion();
+ String newPath = versionedStore.createVersion();
File file = new File(newPath);
FileUtils.writeByteArrayToFile(file, toWrite);
if (toWrite.length != file.length()) {
@@ -260,9 +258,9 @@
+ " bytes to " + file.getCanonicalPath() + ", but "
+ file.length() + " bytes were written.");
}
- _vs.succeedVersion(newPath);
+ versionedStore.succeedVersion(newPath);
if (cleanup) {
- _vs.cleanup(4);
+ versionedStore.cleanup(4);
}
} catch (Exception e) {
throw new RuntimeException(e);
diff --git a/storm-client/src/jvm/org/apache/storm/utils/MutableObject.java b/storm-client/src/jvm/org/apache/storm/utils/MutableObject.java
index 6f165e8..05be6e6 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/MutableObject.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/MutableObject.java
@@ -13,21 +13,20 @@
package org.apache.storm.utils;
public class MutableObject {
- private Object o = null;
+ private Object object = null;
public MutableObject() {
-
}
- public MutableObject(Object o) {
- this.o = o;
+ public MutableObject(Object object) {
+ this.object = object;
}
public synchronized Object getObject() {
- return o;
+ return object;
}
public synchronized void setObject(Object o) {
- this.o = o;
+ this.object = o;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
index 234d849..b0556cd 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
@@ -63,7 +63,7 @@
*/
public NimbusClient(Map<String, Object> conf, String host, int port, Integer timeout) throws TTransportException {
super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, null);
- client = new Nimbus.Client(_protocol);
+ client = new Nimbus.Client(protocol);
isLocal = false;
}
@@ -78,7 +78,7 @@
*/
public NimbusClient(Map<String, Object> conf, String host, Integer port, Integer timeout, String asUser) throws TTransportException {
super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, asUser);
- client = new Nimbus.Client(_protocol);
+ client = new Nimbus.Client(protocol);
isLocal = false;
}
@@ -90,7 +90,7 @@
*/
public NimbusClient(Map<String, Object> conf, String host) throws TTransportException {
super(conf, ThriftConnectionType.NIMBUS, host, null, null, null);
- client = new Nimbus.Client(_protocol);
+ client = new Nimbus.Client(protocol);
isLocal = false;
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java b/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
index c9ddbe7..533f546 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ObjectReader.java
@@ -28,9 +28,11 @@
if (o == null) {
return new ArrayList<String>();
} else if (o instanceof String) {
- return new ArrayList<String>() {{
- add((String) o);
- }};
+ return new ArrayList<String>() {
+ {
+ add((String) o);
+ }
+ };
} else if (o instanceof Collection) {
List<String> answer = new ArrayList<String>();
for (Object v : (Collection) o) {
@@ -49,6 +51,17 @@
return o.toString();
}
+ public static String getString(Object o, String defaultValue) {
+ if (null == o) {
+ return defaultValue;
+ }
+ if (o instanceof String) {
+ return (String) o;
+ } else {
+ throw new IllegalArgumentException("Don't know how to convert " + o + " to String");
+ }
+ }
+
public static Integer getInt(Object o) {
Integer result = getInt(o, null);
if (null == result) {
@@ -62,9 +75,9 @@
return defaultValue;
}
- if (o instanceof Integer ||
- o instanceof Short ||
- o instanceof Byte) {
+ if (o instanceof Integer
+ || o instanceof Short
+ || o instanceof Byte) {
return ((Number) o).intValue();
} else if (o instanceof Long) {
final long l = (Long) o;
@@ -123,15 +136,4 @@
throw new IllegalArgumentException("Don't know how to convert " + o + " to boolean");
}
}
-
- public static String getString(Object o, String defaultValue) {
- if (null == o) {
- return defaultValue;
- }
- if (o instanceof String) {
- return (String) o;
- } else {
- throw new IllegalArgumentException("Don't know how to convert " + o + " to String");
- }
- }
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/RotatingMap.java b/storm-client/src/jvm/org/apache/storm/utils/RotatingMap.java
index ce6e723..6283b50 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/RotatingMap.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/RotatingMap.java
@@ -22,29 +22,29 @@
* Expires keys that have not been updated in the configured number of seconds. The algorithm used will take between expirationSecs and
* expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message.
*
- * get, put, remove, containsKey, and size take O(numBuckets) time to run.
+ * <p>get, put, remove, containsKey, and size take O(numBuckets) time to run.
*
- * The advantage of this design is that the expiration thread only locks the object for O(1) time, meaning the object is essentially always
- * available for gets/puts.
+ * <p>The advantage of this design is that the expiration thread only locks the object for O(1) time, meaning the object
+ * is essentially always available for gets/puts.
*
- * Note: This class is not thread-safe since it does not protect against changes to _buckets while it is being read
+ * <p>Note: This class is not thread-safe since it does not protect against changes to buckets while it is being read
*/
public class RotatingMap<K, V> {
//this default ensures things expire at most 50% past the expiration time
private static final int DEFAULT_NUM_BUCKETS = 3;
- private final LinkedList<HashMap<K, V>> _buckets;
- private final ExpiredCallback<K, V> _callback;
+ private final LinkedList<HashMap<K, V>> buckets;
+ private final ExpiredCallback<K, V> callback;
public RotatingMap(int numBuckets, ExpiredCallback<K, V> callback) {
if (numBuckets < 2) {
throw new IllegalArgumentException("numBuckets must be >= 2");
}
- _buckets = new LinkedList<>();
+ buckets = new LinkedList<>();
for (int i = 0; i < numBuckets; i++) {
- _buckets.add(new HashMap<K, V>());
+ buckets.add(new HashMap<K, V>());
}
- _callback = callback;
+ this.callback = callback;
}
public RotatingMap(ExpiredCallback<K, V> callback) {
@@ -56,18 +56,18 @@
}
public Map<K, V> rotate() {
- Map<K, V> dead = _buckets.removeLast();
- _buckets.addFirst(new HashMap<K, V>());
- if (_callback != null) {
+ Map<K, V> dead = buckets.removeLast();
+ buckets.addFirst(new HashMap<K, V>());
+ if (callback != null) {
for (Entry<K, V> entry : dead.entrySet()) {
- _callback.expire(entry.getKey(), entry.getValue());
+ callback.expire(entry.getKey(), entry.getValue());
}
}
return dead;
}
public boolean containsKey(K key) {
- for (HashMap<K, V> bucket : _buckets) {
+ for (HashMap<K, V> bucket : buckets) {
if (bucket.containsKey(key)) {
return true;
}
@@ -76,7 +76,7 @@
}
public V get(K key) {
- for (HashMap<K, V> bucket : _buckets) {
+ for (HashMap<K, V> bucket : buckets) {
if (bucket.containsKey(key)) {
return bucket.get(key);
}
@@ -85,7 +85,7 @@
}
public void put(K key, V value) {
- Iterator<HashMap<K, V>> it = _buckets.iterator();
+ Iterator<HashMap<K, V>> it = buckets.iterator();
HashMap<K, V> bucket = it.next();
bucket.put(key, value);
while (it.hasNext()) {
@@ -95,7 +95,7 @@
}
public V remove(K key) {
- for (HashMap<K, V> bucket : _buckets) {
+ for (HashMap<K, V> bucket : buckets) {
if (bucket.containsKey(key)) {
return bucket.remove(key);
}
@@ -105,7 +105,7 @@
public int size() {
int size = 0;
- for (HashMap<K, V> bucket : _buckets) {
+ for (HashMap<K, V> bucket : buckets) {
size += bucket.size();
}
return size;
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ShellBoltMessageQueue.java b/storm-client/src/jvm/org/apache/storm/utils/ShellBoltMessageQueue.java
index 2664992..071ef4e 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ShellBoltMessageQueue.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ShellBoltMessageQueue.java
@@ -44,7 +44,7 @@
}
/**
- * put list of task id to its queue
+ * put list of task id to its queue.
*
* @param taskIds task ids that received the tuples
*/
@@ -59,10 +59,9 @@
}
/**
- * put bolt message to its queue
+ * put bolt message to its queue.
*
* @param boltMsg BoltMsg to pass to subprocess
- * @throws InterruptedException
*/
public void putBoltMsg(BoltMsg boltMsg) throws InterruptedException {
boltMsgQueue.put(boltMsg);
@@ -81,10 +80,8 @@
*
* @param timeout how long to wait before giving up, in units of unit
* @param unit a TimeUnit determining how to interpret the timeout parameter
- * @return List\<Integer\> if task id is available, BoltMsg if task id is not available but bolt message is available, null if the
+ * @return {@code List<Integer>} if task id is available, BoltMsg if task id is not available but bolt message is available, null if the
* specified waiting time elapses before an element is available.
- *
- * @throws InterruptedException
*/
public Object poll(long timeout, TimeUnit unit) throws InterruptedException {
takeLock.lockInterruptibly();
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ShellCommandRunner.java b/storm-client/src/jvm/org/apache/storm/utils/ShellCommandRunner.java
index d434960..e1a5878 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ShellCommandRunner.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ShellCommandRunner.java
@@ -59,7 +59,7 @@
throws IOException;
/**
- * Token separator regex used to parse Shell tool outputs
+ * Token separator regex used to parse Shell tool outputs.
*/
String getTokenSeparatorRegex();
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ShellProcess.java b/storm-client/src/jvm/org/apache/storm/utils/ShellProcess.java
index d857b64..8cb98bf 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ShellProcess.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ShellProcess.java
@@ -35,7 +35,7 @@
public ISerializer serializer;
public Number pid;
public String componentName;
- private Process _subprocess;
+ private Process subprocess;
private InputStream processErrorStream;
private String[] command;
private Map<String, String> env = new HashMap<>();
@@ -78,9 +78,9 @@
this.serializer = getSerializer(conf);
try {
- _subprocess = builder.start();
- processErrorStream = _subprocess.getErrorStream();
- serializer.initialize(_subprocess.getOutputStream(), _subprocess.getInputStream());
+ subprocess = builder.start();
+ processErrorStream = subprocess.getErrorStream();
+ serializer.initialize(subprocess.getOutputStream(), subprocess.getInputStream());
this.pid = serializer.connect(conf, context);
} catch (IOException e) {
throw new RuntimeException(
@@ -94,24 +94,24 @@
private ISerializer getSerializer(Map<String, Object> conf) {
//get factory class name
- String serializer_className = (String) conf.get(Config.TOPOLOGY_MULTILANG_SERIALIZER);
- LOG.info("Storm multilang serializer: " + serializer_className);
+ String serializerClassName = (String) conf.get(Config.TOPOLOGY_MULTILANG_SERIALIZER);
+ LOG.info("Storm multilang serializer: " + serializerClassName);
ISerializer serializer;
try {
//create a factory class
- Class klass = Class.forName(serializer_className);
+ Class klass = Class.forName(serializerClassName);
//obtain a serializer object
Object obj = klass.newInstance();
serializer = (ISerializer) obj;
} catch (Exception e) {
- throw new RuntimeException("Failed to construct multilang serializer from serializer " + serializer_className, e);
+ throw new RuntimeException("Failed to construct multilang serializer from serializer " + serializerClassName, e);
}
return serializer;
}
public void destroy() {
- _subprocess.destroy();
+ subprocess.destroy();
}
public ShellMsg readShellMsg() throws IOException {
@@ -149,6 +149,7 @@
ShellLogger.info(new String(errorReadingBuffer));
}
} catch (Exception e) {
+ //ignore
}
}
@@ -172,25 +173,24 @@
}
/**
+ * Get PID.
* @return pid, if the process has been launched, null otherwise.
*/
public Number getPid() {
return this.pid;
}
- /**
- * @return the name of component.
- */
public String getComponentName() {
return this.componentName;
}
/**
+ * Get exit code.
* @return exit code of the process if process is terminated, -1 if process is not started or terminated.
*/
public int getExitCode() {
try {
- return this._subprocess != null ? this._subprocess.exitValue() : -1;
+ return this.subprocess != null ? this.subprocess.exitValue() : -1;
} catch (IllegalThreadStateException e) {
return -1;
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java b/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
index d16d00a..9e141d7 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/ShellUtils.java
@@ -26,7 +26,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-abstract public class ShellUtils {
+public abstract class ShellUtils {
public static final Logger LOG = LoggerFactory.getLogger(ShellUtils.class);
public static final OSType osType = getOSType();
// Helper static vars for each platform
@@ -45,9 +45,9 @@
*/
public static final String TOKEN_SEPARATOR_REGEX
= WINDOWS ? "[|\n\r]" : "[ \t\n\r\f]";
- final private boolean redirectErrorStream; // merge stdout and stderr
+ private final boolean redirectErrorStream; // merge stdout and stderr
/**
- * Time after which the executing script would be timedout
+ * Time after which the executing script would be timed out.
*/
protected long timeOutInterval = 0L;
private long interval; // refresh interval in msec
@@ -57,11 +57,11 @@
private Process process; // sub process used to execute the command
private int exitCode;
/**
- * If or not script timed out
+ * If or not script timed out.
*/
private AtomicBoolean timedOut;
/**
- * If or not script finished executing
+ * If or not script finished executing.
*/
private volatile AtomicBoolean completed;
@@ -74,7 +74,8 @@
}
/**
- * @param interval the minimum duration to wait before re-executing the command.
+ * Creates a new shell utils instance.
+ * @param interval the minimum duration to wait before re-executing the command
*/
public ShellUtils(long interval, boolean redirectErrorStream) {
this.interval = interval;
@@ -82,7 +83,8 @@
this.redirectErrorStream = redirectErrorStream;
}
- static private OSType getOSType() {
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
+ private static OSType getOSType() {
String osName = System.getProperty("os.name");
if (osName.startsWith("Windows")) {
return OSType.OS_TYPE_WIN;
@@ -101,7 +103,7 @@
}
/**
- * a Unix command to get the current user's groups list
+ * a Unix command to get the current user's groups list.
*/
public static String[] getGroupsCommand() {
if (WINDOWS) {
@@ -156,7 +158,7 @@
}
/**
- * get the exit code
+ * get the exit code.
*
* @return the exit code of the process
*/
@@ -165,7 +167,7 @@
}
/**
- * set the environment for the command
+ * set the environment for the command.
*
* @param env Mapping of environment variables
*/
@@ -174,7 +176,7 @@
}
/**
- * set the working directory
+ * set the working directory.
*
* @param dir The directory where the command would be executed
*/
@@ -183,7 +185,7 @@
}
/**
- * check to see if a command needs to be executed and execute if needed
+ * check to see if a command needs to be executed and execute if needed.
*/
protected void run() throws IOException {
if (lastTime + interval > System.currentTimeMillis()) {
@@ -199,7 +201,7 @@
}
/**
- * Run a command
+ * Run a command.
*/
private void runCommand() throws IOException {
ProcessBuilder builder = new ProcessBuilder(getExecString());
@@ -252,6 +254,7 @@
try {
errThread.start();
} catch (IllegalStateException ise) {
+ //ignore
}
try {
parseExecResult(inReader); // parse the output
@@ -310,18 +313,18 @@
}
/**
- * return an array containing the command name & its parameters
+ * return an array containing the command name & its parameters.
*/
protected abstract String[] getExecString();
/**
- * Parse the execution result
+ * Parse the execution result.
*/
protected abstract void parseExecResult(BufferedReader lines)
throws IOException;
/**
- * get the current sub-process executing the given command
+ * get the current sub-process executing the given command.
*
* @return process executing the command
*/
@@ -345,7 +348,10 @@
this.timedOut.set(true);
}
- // OSType detection
+ /**
+ * OSType detection.
+ */
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public enum OSType {
OS_TYPE_LINUX,
OS_TYPE_WIN,
@@ -437,9 +443,9 @@
protected void parseExecResult(BufferedReader lines) throws IOException {
output = new StringBuffer();
char[] buf = new char[512];
- int nRead;
- while ((nRead = lines.read(buf, 0, buf.length)) > 0) {
- output.append(buf, 0, nRead);
+ int read;
+ while ((read = lines.read(buf, 0, buf.length)) > 0) {
+ output.append(buf, 0, read);
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/SimpleVersion.java b/storm-client/src/jvm/org/apache/storm/utils/SimpleVersion.java
index 1e32d0d..3fe2729 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/SimpleVersion.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/SimpleVersion.java
@@ -20,8 +20,8 @@
*/
public class SimpleVersion implements Comparable<SimpleVersion> {
private static final Pattern VERSION_PATTERN = Pattern.compile("^(\\d+)[\\.\\-\\_]+(\\d+).*$");
- private final int _major;
- private final int _minor;
+ private final int major;
+ private final int minor;
public SimpleVersion(String version) {
Matcher m = VERSION_PATTERN.matcher(version);
@@ -36,21 +36,21 @@
maj = Integer.valueOf(m.group(1));
min = Integer.valueOf(m.group(2));
}
- _major = maj;
- _minor = min;
+ major = maj;
+ minor = min;
}
public int getMajor() {
- return _major;
+ return major;
}
public int getMinor() {
- return _minor;
+ return minor;
}
@Override
public int hashCode() {
- return (Integer.hashCode(_major) * 17) & Integer.hashCode(_minor);
+ return (Integer.hashCode(major) * 17) & Integer.hashCode(minor);
}
@Override
@@ -68,15 +68,15 @@
@Override
public int compareTo(SimpleVersion o) {
- int ret = Integer.compare(_major, o._major);
+ int ret = Integer.compare(major, o.major);
if (ret == 0) {
- ret = Integer.compare(_minor, o._minor);
+ ret = Integer.compare(minor, o.minor);
}
return ret;
}
@Override
public String toString() {
- return _major + "." + _minor;
+ return major + "." + minor;
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/StormBoundedExponentialBackoffRetry.java b/storm-client/src/jvm/org/apache/storm/utils/StormBoundedExponentialBackoffRetry.java
index 969a9c6..1cb40c9 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/StormBoundedExponentialBackoffRetry.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/StormBoundedExponentialBackoffRetry.java
@@ -28,7 +28,7 @@
* The class provides generic exponential-linear backoff retry strategy for storm. It calculates threshold for exponentially increasing
* sleeptime for retries. Beyond this threshold, the sleeptime increase is linear.
*
- * Also adds jitter for exponential/linear retry. It guarantees `currSleepTimeMs >= prevSleepTimeMs` and `baseSleepTimeMs <=
+ * <p>Also adds jitter for exponential/linear retry. It guarantees `currSleepTimeMs >= prevSleepTimeMs` and `baseSleepTimeMs <=
* currSleepTimeMs <= maxSleepTimeMs`
*/
public StormBoundedExponentialBackoffRetry(int baseSleepTimeMs, int maxSleepTimeMs, int maxRetries) {
diff --git a/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java b/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java
index 64d5ace..c1ddb59 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/SupervisorClient.java
@@ -22,7 +22,7 @@
import org.slf4j.LoggerFactory;
/**
- * Client for interacting with Supervisor server, now we use supervisor server mainly for cases below:
+ * Client for interacting with Supervisor server, now we use supervisor server mainly for cases below.
* <ul>
* <li>worker <- supervisor: get worker local assignment for a storm.</li>
* <li>nimbus -> supervisor: assign assignments for a node.</li>
@@ -38,17 +38,17 @@
public SupervisorClient(Map conf, String host, int port, Integer timeout) throws TTransportException {
super(conf, ThriftConnectionType.SUPERVISOR, host, port, timeout, null);
- client = new Supervisor.Client(_protocol);
+ client = new Supervisor.Client(protocol);
}
public SupervisorClient(Map conf, String host, Integer port, Integer timeout, String asUser) throws TTransportException {
super(conf, ThriftConnectionType.SUPERVISOR, host, port, timeout, asUser);
- client = new Supervisor.Client(_protocol);
+ client = new Supervisor.Client(protocol);
}
public SupervisorClient(Map conf, String host) throws TTransportException {
super(conf, ThriftConnectionType.SUPERVISOR, host, null, null, null);
- client = new Supervisor.Client(_protocol);
+ client = new Supervisor.Client(protocol);
}
public static SupervisorClient getConfiguredClient(Map conf, String host) {
@@ -64,8 +64,9 @@
public static SupervisorClient getConfiguredClientAs(Map conf, String host, int port, String asUser) {
if (conf.containsKey(Config.STORM_DO_AS_USER)) {
if (asUser != null && !asUser.isEmpty()) {
- LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence."
- , asUser, conf.get(Config.STORM_DO_AS_USER));
+ LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence.",
+ asUser,
+ conf.get(Config.STORM_DO_AS_USER));
}
asUser = (String) conf.get(Config.STORM_DO_AS_USER);
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/TimeCacheMap.java b/storm-client/src/jvm/org/apache/storm/utils/TimeCacheMap.java
index 429f3d2..7d85219 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/TimeCacheMap.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/TimeCacheMap.java
@@ -19,51 +19,51 @@
* Expires keys that have not been updated in the configured number of seconds. The algorithm used will take between expirationSecs and
* expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message.
*
- * get, put, remove, containsKey, and size take O(numBuckets) time to run.
+ * <p>get, put, remove, containsKey, and size take O(numBuckets) time to run.
*
- * The advantage of this design is that the expiration thread only locks the object for O(1) time, meaning the object is essentially always
- * available for gets/puts.
+ * <p>The advantage of this design is that the expiration thread only locks the object for O(1) time, meaning the object
+ * is essentially always available for gets/puts.
*/
//deprecated in favor of non-threaded RotatingMap
@Deprecated
public class TimeCacheMap<K, V> {
//this default ensures things expire at most 50% past the expiration time
private static final int DEFAULT_NUM_BUCKETS = 3;
- private final RotatingMap<K, V> _rotatingMap;
- private final Object _lock = new Object();
- private final Thread _cleaner;
- private ExpiredCallback<K, V> _callback;
+ private final RotatingMap<K, V> rotatingMap;
+ private final Object lock = new Object();
+ private final Thread cleaner;
+ private ExpiredCallback<K, V> callback;
public TimeCacheMap(int expirationSecs, int numBuckets, ExpiredCallback<K, V> callback) {
- _rotatingMap = new RotatingMap<>(numBuckets);
+ rotatingMap = new RotatingMap<>(numBuckets);
- _callback = callback;
+ this.callback = callback;
final long expirationMillis = expirationSecs * 1000L;
final long sleepTime = expirationMillis / (numBuckets - 1);
- _cleaner = new Thread(new Runnable() {
+ cleaner = new Thread(new Runnable() {
@Override
public void run() {
try {
while (true) {
Map<K, V> dead = null;
Time.sleep(sleepTime);
- synchronized (_lock) {
- dead = _rotatingMap.rotate();
+ synchronized (lock) {
+ dead = rotatingMap.rotate();
}
- if (_callback != null) {
+ if (TimeCacheMap.this.callback != null) {
for (Entry<K, V> entry : dead.entrySet()) {
- _callback.expire(entry.getKey(), entry.getValue());
+ TimeCacheMap.this.callback.expire(entry.getKey(), entry.getValue());
}
}
}
} catch (InterruptedException ex) {
-
+ //ignore
}
}
});
- _cleaner.setDaemon(true);
- _cleaner.start();
+ cleaner.setDaemon(true);
+ cleaner.start();
}
public TimeCacheMap(int expirationSecs, ExpiredCallback<K, V> callback) {
@@ -79,37 +79,37 @@
}
public boolean containsKey(K key) {
- synchronized (_lock) {
- return _rotatingMap.containsKey(key);
+ synchronized (lock) {
+ return rotatingMap.containsKey(key);
}
}
public V get(K key) {
- synchronized (_lock) {
- return _rotatingMap.get(key);
+ synchronized (lock) {
+ return rotatingMap.get(key);
}
}
public void put(K key, V value) {
- synchronized (_lock) {
- _rotatingMap.put(key, value);
+ synchronized (lock) {
+ rotatingMap.put(key, value);
}
}
public Object remove(K key) {
- synchronized (_lock) {
- return _rotatingMap.remove(key);
+ synchronized (lock) {
+ return rotatingMap.remove(key);
}
}
public int size() {
- synchronized (_lock) {
- return _rotatingMap.size();
+ synchronized (lock) {
+ return rotatingMap.size();
}
}
public void cleanup() {
- _cleaner.interrupt();
+ cleaner.interrupt();
}
public interface ExpiredCallback<K, V> {
diff --git a/storm-client/src/jvm/org/apache/storm/utils/Utils.java b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
index 8d15fc2..eb31eaf 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/Utils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
@@ -203,6 +203,10 @@
}
}
+ public static Map<String, Object> findAndReadConfigFile(String name) {
+ return findAndReadConfigFile(name, true);
+ }
+
private static InputStream getConfigFileInputStream(String configFilePath)
throws IOException {
if (null == configFilePath) {
@@ -360,7 +364,7 @@
* Creates a thread that calls the given code repeatedly, sleeping for an interval of seconds equal to the return value of the previous
* call.
*
- * The given afn may be a callable that returns the number of seconds to sleep, or it may be a Callable that returns another Callable
+ * <p>The given afn may be a callable that returns the number of seconds to sleep, or it may be a Callable that returns another Callable
* that in turn returns the number of seconds to sleep. In the latter case isFactory.
*
* @param afn the code to call on each iteration
@@ -587,6 +591,7 @@
* @param conf the config to get the super User ACL from
* @return the super user ACL.
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static ACL getSuperUserAcl(Map<String, Object> conf) {
String stormZKUser = (String) conf.get(Config.STORM_ZOOKEEPER_SUPERACL);
if (stormZKUser == null) {
@@ -601,6 +606,7 @@
* @param conf the config for the topology.
* @return the ACLs
*/
+ @SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public static List<ACL> getWorkerACL(Map<String, Object> conf) {
if (!isZkAuthenticationConfiguredTopology(conf)) {
return null;
@@ -712,12 +718,13 @@
}
/**
- * "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}"
+ * <code>"{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}"</code>.
*
- * Example usage in java: Map<Integer, String> tasks; Map<String, List<Integer>> componentTasks = Utils.reverse_map(tasks);
+ * <p>Example usage in java:
+ * <code>Map<Integer, String> tasks; Map<String, List<Integer>> componentTasks = Utils.reverse_map(tasks);</code>
*
- * The order of he resulting list values depends on the ordering properties of the Map passed in. The caller is responsible for passing
- * an ordered map if they expect the result to be consistently ordered as well.
+ * <p>The order of he resulting list values depends on the ordering properties of the Map passed in. The caller is
+ * responsible for passing an ordered map if they expect the result to be consistently ordered as well.
*
* @param map to reverse
* @return a reversed map
@@ -740,6 +747,30 @@
return rtn;
}
+ /**
+ * "[[:a 1] [:b 1] [:c 2]} -> {1 [:a :b] 2 :c}" Reverses an assoc-list style Map like reverseMap(Map...)
+ *
+ * @param listSeq to reverse
+ * @return a reversed map
+ */
+ public static Map<Object, List<Object>> reverseMap(List<List<Object>> listSeq) {
+ Map<Object, List<Object>> rtn = new HashMap<>();
+ if (listSeq == null) {
+ return rtn;
+ }
+ for (List<Object> listEntry : listSeq) {
+ Object key = listEntry.get(0);
+ Object val = listEntry.get(1);
+ List<Object> list = rtn.get(val);
+ if (list == null) {
+ list = new ArrayList<>();
+ rtn.put(val, list);
+ }
+ list.add(key);
+ }
+ return rtn;
+ }
+
public static boolean isOnWindows() {
if (System.getenv("OS") != null) {
return System.getenv("OS").equals("Windows_NT");
@@ -933,6 +964,7 @@
}
/**
+ * Get process PID.
* @return the pid of this JVM, because Java doesn't provide a real way to do this.
*/
public static String processPid() {
@@ -991,39 +1023,9 @@
Thread.setDefaultUncaughtExceptionHandler(createDefaultUncaughtExceptionHandler());
}
- public static Map<String, Object> findAndReadConfigFile(String name) {
- return findAndReadConfigFile(name, true);
- }
-
- /**
- * "[[:a 1] [:b 1] [:c 2]} -> {1 [:a :b] 2 :c}" Reverses an assoc-list style Map like reverseMap(Map...)
- *
- * @param listSeq to reverse
- * @return a reversed map
- */
- public static Map<Object, List<Object>> reverseMap(List<List<Object>> listSeq) {
- Map<Object, List<Object>> rtn = new HashMap<>();
- if (listSeq == null) {
- return rtn;
- }
- for (List<Object> listEntry : listSeq) {
- Object key = listEntry.get(0);
- Object val = listEntry.get(1);
- List<Object> list = rtn.get(val);
- if (list == null) {
- list = new ArrayList<>();
- rtn.put(val, list);
- }
- list.add(key);
- }
- return rtn;
- }
-
/**
* parses the arguments to extract jvm heap memory size in MB.
*
- * @param options
- * @param defaultValue
* @return the value of the JVM heap memory setting (in MB) in a java command.
*/
public static Double parseJvmHeapMemByChildOpts(List<String> options, Double defaultValue) {
@@ -1168,8 +1170,6 @@
* Validate topology blobstore map.
*
* @param topoConf Topology configuration
- * @throws InvalidTopologyException
- * @throws AuthorizationException
*/
public static void validateTopologyBlobStoreMap(Map<String, Object> topoConf) throws InvalidTopologyException, AuthorizationException {
try (NimbusBlobStore client = new NimbusBlobStore()) {
@@ -1183,8 +1183,6 @@
*
* @param topoConf Topology configuration
* @param client The NimbusBlobStore client. It must call prepare() before being used here.
- * @throws InvalidTopologyException
- * @throws AuthorizationException
*/
public static void validateTopologyBlobStoreMap(Map<String, Object> topoConf, NimbusBlobStore client)
throws InvalidTopologyException, AuthorizationException {
@@ -1205,11 +1203,6 @@
/**
* Validate topology blobstore map.
- *
- * @param topoConf Topology configuration
- * @param blobStore The BlobStore
- * @throws InvalidTopologyException
- * @throws AuthorizationException
*/
public static void validateTopologyBlobStoreMap(Map<String, Object> topoConf, BlobStore blobStore)
throws InvalidTopologyException, AuthorizationException {
@@ -1234,11 +1227,11 @@
*/
public static String threadDump() {
final StringBuilder dump = new StringBuilder();
- final java.lang.management.ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean();
- final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100);
+ final java.lang.management.ThreadMXBean threadMxBean = ManagementFactory.getThreadMXBean();
+ final java.lang.management.ThreadInfo[] threadInfos = threadMxBean.getThreadInfo(threadMxBean.getAllThreadIds(), 100);
for (Entry<Thread, StackTraceElement[]> entry: Thread.getAllStackTraces().entrySet()) {
Thread t = entry.getKey();
- ThreadInfo threadInfo = threadMXBean.getThreadInfo(t.getId());
+ ThreadInfo threadInfo = threadMxBean.getThreadInfo(t.getId());
if (threadInfo == null) {
//Thread died before we could get the info, skip
continue;
@@ -1321,6 +1314,7 @@
* @param b something else
* @return a or b the first one that is not null
*/
+ @SuppressWarnings({"checkstyle:AbbreviationAsWordInName", "checkstyle:MethodName"})
public static <V> V OR(V a, V b) {
return a == null ? b : a;
}
@@ -1338,9 +1332,9 @@
}
/**
- * Fills up chunks out of a collection (given a maximum amount of chunks)
+ * Fills up chunks out of a collection (given a maximum amount of chunks).
*
- * i.e. partitionFixed(5, [1,2,3]) -> [[1,2,3]] partitionFixed(5, [1..9]) -> [[1,2], [3,4], [5,6], [7,8], [9]] partitionFixed(3,
+ * <p>i.e. partitionFixed(5, [1,2,3]) -> [[1,2,3]] partitionFixed(5, [1..9]) -> [[1,2], [3,4], [5,6], [7,8], [9]] partitionFixed(3,
* [1..10]) -> [[1,2,3,4], [5,6,7], [8,9,10]]
*
* @param maxNumChunks the maximum number of chunks to return
@@ -1398,7 +1392,6 @@
* Gets an available port. Consider if it is possible to pass port 0 to the server instead of using this method, since there is no
* guarantee that the port returned by this method will remain free.
*
- * @param preferredPort
* @return The preferred port if available, or a random available port
*/
public static int getAvailablePort(int preferredPort) {
@@ -1414,7 +1407,7 @@
}
/**
- * Shortcut to calling {@link #getAvailablePort(int) } with 0 as the preferred port
+ * Shortcut to calling {@link #getAvailablePort(int) } with 0 as the preferred port.
*
* @return A random available port
*/
@@ -1492,18 +1485,18 @@
* Get a map of version to classpath from the conf Config.SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP
*
* @param conf what to read it out of
- * @param currentCP the current classpath for this version of storm (not included in the conf, but returned by this)
+ * @param currentClassPath the current classpath for this version of storm (not included in the conf, but returned by this)
* @return the map
*/
public static NavigableMap<SimpleVersion, List<String>> getConfiguredClasspathVersions(Map<String, Object> conf,
- List<String> currentCP) {
+ List<String> currentClassPath) {
TreeMap<SimpleVersion, List<String>> ret = new TreeMap<>();
Map<String, String> fromConf =
(Map<String, String>) conf.getOrDefault(Config.SUPERVISOR_WORKER_VERSION_CLASSPATH_MAP, Collections.emptyMap());
for (Map.Entry<String, String> entry : fromConf.entrySet()) {
ret.put(new SimpleVersion(entry.getKey()), Arrays.asList(entry.getValue().split(File.pathSeparator)));
}
- ret.put(VersionInfo.OUR_VERSION, currentCP);
+ ret.put(VersionInfo.OUR_VERSION, currentClassPath);
return ret;
}
@@ -1734,6 +1727,7 @@
try {
FileUtils.forceDelete(new File(path));
} catch (FileNotFoundException ignored) {
+ //ignore
}
}
}
@@ -1804,13 +1798,13 @@
private Yaml yaml;
private Map<String, Object> defaultsConf;
private Map<String, Object> stormConf;
- private File f;
+ private File file;
- public JarConfigReader(Yaml yaml, Map<String, Object> defaultsConf, Map<String, Object> stormConf, File f) {
+ public JarConfigReader(Yaml yaml, Map<String, Object> defaultsConf, Map<String, Object> stormConf, File file) {
this.yaml = yaml;
this.defaultsConf = defaultsConf;
this.stormConf = stormConf;
- this.f = f;
+ this.file = file;
}
public Map<String, Object> getDefaultsConf() {
@@ -1822,14 +1816,14 @@
}
public JarConfigReader readZip() throws IOException {
- try (ZipFile zipFile = new ZipFile(f)) {
+ try (ZipFile zipFile = new ZipFile(file)) {
readArchive(zipFile);
}
return this;
}
public JarConfigReader readJar() throws IOException {
- try (JarFile jarFile = new JarFile(f)) {
+ try (JarFile jarFile = new JarFile(file)) {
readArchive(jarFile);
}
return this;
diff --git a/storm-client/src/jvm/org/apache/storm/utils/VersionInfo.java b/storm-client/src/jvm/org/apache/storm/utils/VersionInfo.java
index bd8ff51..c71a58c 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/VersionInfo.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/VersionInfo.java
@@ -100,10 +100,10 @@
@Override
public String getBuildVersion() {
- return this.getVersion() +
- " from " + getRevision() +
- " by " + getUser() +
- " source checksum " + getSrcChecksum();
+ return this.getVersion()
+ + " from " + getRevision()
+ + " by " + getUser()
+ + " source checksum " + getSrcChecksum();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/VersionedStore.java b/storm-client/src/jvm/org/apache/storm/utils/VersionedStore.java
index 329ebbc..08d09f8 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/VersionedStore.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/VersionedStore.java
@@ -23,7 +23,7 @@
public class VersionedStore {
private static final String FINISHED_VERSION_SUFFIX = ".version";
- private String _root;
+ private String root;
/**
* Creates a store at the given path.
@@ -32,18 +32,18 @@
* @param createRootDir option to create the path directory
*/
public VersionedStore(String path, boolean createRootDir) throws IOException {
- _root = path;
+ root = path;
if (createRootDir) {
- mkdirs(_root);
+ mkdirs(root);
}
}
public String getRoot() {
- return _root;
+ return root;
}
public String versionPath(long version) {
- return new File(_root, "" + version).getAbsolutePath();
+ return new File(root, "" + version).getAbsolutePath();
}
public String mostRecentVersionPath() throws IOException {
@@ -132,7 +132,7 @@
}
HashSet<Long> keepers = new HashSet<Long>(versions);
- for (String p : listDir(_root)) {
+ for (String p : listDir(root)) {
Long v = parseVersion(p);
if (v != null && !keepers.contains(v)) {
deleteVersion(v);
@@ -145,7 +145,7 @@
*/
public List<Long> getAllVersions() throws IOException {
List<Long> ret = new ArrayList<Long>();
- for (String s : listDir(_root)) {
+ for (String s : listDir(root)) {
if (s.endsWith(FINISHED_VERSION_SUFFIX) && new File(s.substring(0, s.length() - FINISHED_VERSION_SUFFIX.length())).exists()) {
ret.add(validateAndGetVersion(s));
@@ -157,7 +157,7 @@
}
private String tokenPath(long version) {
- return new File(_root, "" + version + FINISHED_VERSION_SUFFIX).getAbsolutePath();
+ return new File(root, "" + version + FINISHED_VERSION_SUFFIX).getAbsolutePath();
}
private long validateAndGetVersion(String path) {
diff --git a/storm-client/src/jvm/org/apache/storm/utils/WindowedTimeThrottler.java b/storm-client/src/jvm/org/apache/storm/utils/WindowedTimeThrottler.java
index 3b031d1..5b6d9b5 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/WindowedTimeThrottler.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/WindowedTimeThrottler.java
@@ -13,34 +13,34 @@
package org.apache.storm.utils;
public class WindowedTimeThrottler {
- long _windowMillis;
- int _maxAmt;
- long _windowStartTime;
- int _windowEvents = 0;
+ long windowMillis;
+ int maxAmt;
+ long windowStartTime;
+ int windowEvents = 0;
public WindowedTimeThrottler(Number windowMillis, Number maxAmt) {
- _windowMillis = windowMillis.longValue();
- _maxAmt = maxAmt.intValue();
- _windowStartTime = System.currentTimeMillis();
+ this.windowMillis = windowMillis.longValue();
+ this.maxAmt = maxAmt.intValue();
+ windowStartTime = System.currentTimeMillis();
}
public boolean isThrottled() {
resetIfNecessary();
- return _windowEvents >= _maxAmt;
+ return windowEvents >= maxAmt;
}
//returns void if the event should continue, false if the event should not be done
public void markEvent() {
resetIfNecessary();
- _windowEvents++;
+ windowEvents++;
}
private void resetIfNecessary() {
long now = System.currentTimeMillis();
- if (now - _windowStartTime > _windowMillis) {
- _windowStartTime = now;
- _windowEvents = 0;
+ if (now - windowStartTime > windowMillis) {
+ windowStartTime = now;
+ windowEvents = 0;
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/WrappedDRPCExecutionException.java b/storm-client/src/jvm/org/apache/storm/utils/WrappedDRPCExecutionException.java
index afa01a0..0aa5122 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/WrappedDRPCExecutionException.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/WrappedDRPCExecutionException.java
@@ -23,7 +23,9 @@
/**
* Wraps the generated TException to allow getMessage() to return a valid string.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class WrappedDRPCExecutionException extends DRPCExecutionException {
+
public WrappedDRPCExecutionException(String msg) {
super(msg);
}
diff --git a/storm-client/src/jvm/org/apache/storm/utils/WrappedHBExecutionException.java b/storm-client/src/jvm/org/apache/storm/utils/WrappedHBExecutionException.java
index ab976c0..85662ea 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/WrappedHBExecutionException.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/WrappedHBExecutionException.java
@@ -23,6 +23,7 @@
/**
* Wraps the generated TException to allow getMessage() to return a valid string.
*/
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class WrappedHBExecutionException extends HBExecutionException {
public WrappedHBExecutionException(String msg) {
super(msg);
diff --git a/storm-client/src/jvm/org/apache/storm/utils/WritableUtils.java b/storm-client/src/jvm/org/apache/storm/utils/WritableUtils.java
index 3db1805..e21fc4b 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/WritableUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/WritableUtils.java
@@ -113,8 +113,9 @@
}
}
-
- /* Ugly utility, maybe someone else can do this better */
+ /**
+ * Ugly utility, maybe someone else can do this better.
+ */
public static String readCompressedString(DataInput in) throws IOException {
byte[] bytes = readCompressedByteArray(in);
if (bytes == null) {
@@ -128,12 +129,10 @@
return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null);
}
- /*
- *
+ /**
* Write a String as a Network Int n, followed by n Bytes
* Alternative to 16 bit read/writeUTF.
* Encoding standard is... ?
- *
*/
public static void writeString(DataOutput out, String s) throws IOException {
if (s != null) {
@@ -146,11 +145,10 @@
}
}
- /*
+ /**
* Read a String as a Network Int n, followed by n Bytes
* Alternative to 16 bit read/writeUTF.
* Encoding standard is... ?
- *
*/
public static String readString(DataInput in) throws IOException {
int length = in.readInt();
@@ -163,10 +161,9 @@
}
- /*
+ /**
* Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
* Could be generalised using introspection.
- *
*/
public static void writeStringArray(DataOutput out, String[] s) throws IOException {
out.writeInt(s.length);
@@ -192,10 +189,9 @@
}
}
- /*
+ /**
* Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
* Could be generalised using introspection. Actually this bit couldn't...
- *
*/
public static String[] readStringArray(DataInput in) throws IOException {
int len = in.readInt();
@@ -210,10 +206,9 @@
}
- /*
+ /**
* Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
* Could be generalised using introspection. Handles null arrays and null values.
- *
*/
public static String[] readCompressedStringArray(DataInput in) throws IOException {
int len = in.readInt();
@@ -228,10 +223,8 @@
}
- /*
- *
+ /**
* Test Utility Method Display Byte Array.
- *
*/
public static void displayByteArray(byte[] record) {
int i;
@@ -262,7 +255,6 @@
*
* @param stream Binary output stream
* @param i Integer to be serialized
- * @throws java.io.IOException
*/
public static void writeVInt(DataOutput stream, int i) throws IOException {
writeVLong(stream, i);
@@ -281,7 +273,6 @@
*
* @param stream Binary output stream
* @param i Long to be serialized
- * @throws java.io.IOException
*/
public static void writeVLong(DataOutput stream, long i) throws IOException {
if (i >= -112 && i <= 127) {
@@ -316,7 +307,6 @@
/**
* Reads a zero-compressed encoded long from input stream and returns it.
* @param stream Binary input stream
- * @throws java.io.IOException
* @return deserialized long from stream.
*/
public static long readVLong(DataInput stream) throws IOException {
@@ -337,7 +327,6 @@
/**
* Reads a zero-compressed encoded integer from input stream and returns it.
* @param stream Binary input stream
- * @throws java.io.IOException
* @return deserialized integer from stream.
*/
public static int readVInt(DataInput stream) throws IOException {
@@ -345,7 +334,7 @@
}
/**
- * Given the first byte of a vint/vlong, determine the sign
+ * Given the first byte of a vint/vlong, determine the sign.
* @param value the first byte
* @return is the value negative
*/
@@ -354,7 +343,7 @@
}
/**
- * Parse the first byte of a vint/vlong to determine the number of bytes
+ * Parse the first byte of a vint/vlong to determine the number of bytes.
* @param value the first byte of the vint/vlong
* @return the total number of bytes (1 to 9)
*/
@@ -368,7 +357,7 @@
}
/**
- * Get the encoded length if an integer is stored in a variable-length format
+ * Get the encoded length if an integer is stored in a variable-length format.
* @return the encoded length
*/
public static int getVIntSize(long i) {
@@ -386,7 +375,7 @@
}
/**
- * Skip <i>len</i> number of bytes in input stream<i>in</i>
+ * Skip <i>len</i> number of bytes in input stream<i>in</i>.
* @param in input stream
* @param len number of bytes to skip
* @throws IOException when skipped less number of bytes
@@ -400,8 +389,10 @@
}
if (total < len) {
- throw new IOException("Not able to skip " + len + " bytes, possibly " +
- "due to end of input.");
+ throw new IOException("Not able to skip "
+ + len
+ + " bytes, possibly "
+ + "due to end of input.");
}
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
index b7b2458..f5db47c 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@ -52,16 +52,6 @@
private static final int DESIRED_FIELD_ACC = ACC_PUBLIC | ACC_STATIC | ACC_FINAL;
private static List<Class<?>> configClasses = null;
- /**
- * Validates a field given field name as string uses Config.java as the default config class
- *
- * @param fieldName provided as a string
- * @param conf map of confs
- */
- public static void validateField(String fieldName, Map<String, Object> conf) {
- validateField(fieldName, conf, getConfigClasses());
- }
-
public static synchronized List<Class<?>> getConfigClasses() {
if (configClasses == null) {
List<Class<?>> ret = new ArrayList<>();
@@ -96,6 +86,16 @@
}
/**
+ * Validates a field given field name as string uses Config.java as the default config class
+ *
+ * @param fieldName provided as a string
+ * @param conf map of confs
+ */
+ public static void validateField(String fieldName, Map<String, Object> conf) {
+ validateField(fieldName, conf, getConfigClasses());
+ }
+
+ /**
* Validates a field given field name as string.
*
* @param fieldName provided as a string
@@ -187,12 +187,6 @@
validateFields(conf, getConfigClasses());
}
- public static boolean isFieldAllowed(Field field) {
- return field.getAnnotation(NotConf.class) == null
- && String.class.equals(field.getType())
- && ((field.getModifiers() & DESIRED_FIELD_ACC) == DESIRED_FIELD_ACC) && !field.isSynthetic();
- }
-
/**
* Validate all confs in map.
*
@@ -223,6 +217,12 @@
}
}
+ public static boolean isFieldAllowed(Field field) {
+ return field.getAnnotation(NotConf.class) == null
+ && String.class.equals(field.getType())
+ && ((field.getModifiers() & DESIRED_FIELD_ACC) == DESIRED_FIELD_ACC) && !field.isSynthetic();
+ }
+
private static Map<String, Object> getParamsFromAnnotation(Class<?> validatorClass, Object v)
throws InvocationTargetException, IllegalAccessException {
Map<String, Object> params = new HashMap<String, Object>();
@@ -304,7 +304,7 @@
}
/**
- * Checks if the named type derives from the specified Class
+ * Checks if the named type derives from the specified Class.
*/
public static class DerivedTypeValidator extends Validator {
@@ -586,8 +586,8 @@
((Validator) v).validateField(name + " list entry", entry);
} else {
LOG.warn(
- "validator: {} cannot be used in ListEntryCustomValidator. Individual entry validators must a instance of " +
- "Validator class",
+ "validator: {} cannot be used in ListEntryCustomValidator. Individual entry validators must a instance of "
+ + "Validator class",
validator.getName());
}
}
@@ -656,8 +656,8 @@
((Validator) keyValidator).validateField(name + " Map key", entry.getKey());
} else {
LOG.warn(
- "validator: {} cannot be used in MapEntryCustomValidator to validate keys. Individual entry validators must " +
- "a instance of Validator class",
+ "validator: {} cannot be used in MapEntryCustomValidator to validate keys. Individual entry validators must "
+ + "a instance of Validator class",
kv.getName());
}
}
@@ -667,8 +667,8 @@
((Validator) valueValidator).validateField(name + " Map value", entry.getValue());
} else {
LOG.warn(
- "validator: {} cannot be used in MapEntryCustomValidator to validate values. Individual entry validators " +
- "must a instance of Validator class",
+ "validator: {} cannot be used in MapEntryCustomValidator to validate values. Individual entry validators "
+ + "must a instance of Validator class",
vv.getName());
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationAnnotations.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationAnnotations.java
index 6f00441..ac8ba9d 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationAnnotations.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationAnnotations.java
@@ -26,11 +26,11 @@
*/
public class ConfigValidationAnnotations {
/**
- * Validators with fields: validatorClass and type
+ * Validators with fields: validatorClass and type.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isType {
+ public @interface IsType {
Class<?> validatorClass() default ConfigValidation.SimpleTypeValidator.class;
Class<?> type();
@@ -38,7 +38,7 @@
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isDerivedFrom {
+ public @interface IsDerivedFrom {
Class<?> validatorClass() default ConfigValidation.DerivedTypeValidator.class;
Class<?> baseType();
@@ -46,29 +46,29 @@
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isStringList {
+ public @interface IsStringList {
Class<?> validatorClass() default ConfigValidation.ListEntryTypeValidator.class;
Class<?> type() default String.class;
}
/**
- * validates each entry in a list is of a certain type
+ * validates each entry in a list is of a certain type.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isListEntryType {
+ public @interface IsListEntryType {
Class<?> validatorClass() default ConfigValidation.ListEntryTypeValidator.class;
Class<?> type();
}
/**
- * Validators with fields: validatorClass
+ * Validators with fields: validatorClass.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isString {
+ public @interface IsString {
Class<?> validatorClass() default ConfigValidation.StringValidator.class;
String[] acceptedValues() default "";
@@ -76,24 +76,24 @@
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isNumber {
+ public @interface IsNumber {
Class<?> validatorClass() default ConfigValidation.NumberValidator.class;
}
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isBoolean {
+ public @interface IsBoolean {
Class<?> validatorClass() default ConfigValidation.BooleanValidator.class;
}
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isInteger {
+ public @interface IsInteger {
Class<?> validatorClass() default ConfigValidation.IntegerValidator.class;
}
/**
- * Validates on object is not null
+ * Validates on object is not null.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
@@ -102,31 +102,31 @@
}
/**
- * Validates that there are no duplicates in a list
+ * Validates that there are no duplicates in a list.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isNoDuplicateInList {
+ public @interface IsNoDuplicateInList {
Class<?> validatorClass() default ConfigValidation.NoDuplicateInListValidator.class;
}
/**
- * Validates each entry in a list with a list of validators Validators with fields: validatorClass and entryValidatorClass
+ * Validates each entry in a list with a list of validators Validators with fields: validatorClass and entryValidatorClass.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isListEntryCustom {
+ public @interface IsListEntryCustom {
Class<?> validatorClass() default ConfigValidation.ListEntryCustomValidator.class;
Class<?>[] entryValidatorClasses();
}
/**
- * Validates the type of each key and value in a map Validator with fields: validatorClass, keyValidatorClass, valueValidatorClass
+ * Validates the type of each key and value in a map Validator with fields: validatorClass, keyValidatorClass, valueValidatorClass.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isMapEntryType {
+ public @interface IsMapEntryType {
Class<?> validatorClass() default ConfigValidation.MapEntryTypeValidator.class;
Class<?> keyType();
@@ -136,11 +136,11 @@
/**
* Validates a each key and value in a Map with a list of validators Validator with fields: validatorClass, keyValidatorClasses,
- * valueValidatorClasses
+ * valueValidatorClasses.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isMapEntryCustom {
+ public @interface IsMapEntryCustom {
Class<?> validatorClass() default ConfigValidation.MapEntryCustomValidator.class;
Class<?>[] keyValidatorClasses();
@@ -149,11 +149,11 @@
}
/**
- * Checks if a number is positive and whether zero inclusive Validator with fields: validatorClass, includeZero
+ * Checks if a number is positive and whether zero inclusive Validator with fields: validatorClass, includeZero.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isPositiveNumber {
+ public @interface IsPositiveNumber {
Class<?> validatorClass() default ConfigValidation.PositiveNumberValidator.class;
boolean includeZero() default false;
@@ -161,35 +161,35 @@
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isImplementationOfClass {
+ public @interface IsImplementationOfClass {
Class<?> validatorClass() default ConfigValidation.ImplementsClassValidator.class;
Class<?> implementsClass();
}
/**
- * Complex/custom type validators
+ * Complex/custom type validators.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isStringOrStringList {
+ public @interface IsStringOrStringList {
Class<?> validatorClass() default ConfigValidation.StringOrStringListValidator.class;
}
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isKryoReg {
+ public @interface IsKryoReg {
Class<?> validatorClass() default ConfigValidation.KryoRegValidator.class;
}
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
- public @interface isPowerOf2 {
+ public @interface IsPowerOf2 {
Class<?> validatorClass() default ConfigValidation.PowerOf2Validator.class;
}
/**
- * For custom validators
+ * For custom validators.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.FIELD)
@@ -204,7 +204,7 @@
}
/**
- * Field names for annotations
+ * Field names for annotations.
*/
public static class ValidatorParams {
static final String VALIDATOR_CLASS = "validatorClass";
diff --git a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationUtils.java b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationUtils.java
index 9a9f22c..1510a1b 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationUtils.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/ConfigValidationUtils.java
@@ -84,9 +84,10 @@
}
return;
}
- throw new IllegalArgumentException(
- "Field " + name + " must be an Iterable but was " +
- ((field == null) ? "null" : ("a " + field.getClass())));
+ throw new IllegalArgumentException("Field "
+ + name
+ + " must be an Iterable but was "
+ + ((field == null) ? "null" : ("a " + field.getClass())));
}
};
}
@@ -156,7 +157,7 @@
/**
* Declares a method for validating configuration values that is nestable.
*/
- public static abstract class NestableFieldValidator implements FieldValidator {
+ public abstract static class NestableFieldValidator implements FieldValidator {
@Override
public void validateField(String name, Object field) throws IllegalArgumentException {
validateField(null, name, field);
diff --git a/storm-client/src/jvm/org/apache/storm/validation/NotConf.java b/storm-client/src/jvm/org/apache/storm/validation/NotConf.java
index 8f7ff6a..50063f1 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/NotConf.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/NotConf.java
@@ -13,7 +13,7 @@
package org.apache.storm.validation;
/**
- * Annotation that can be used to explicitly call out public static final String fields that are not configs
+ * Annotation that can be used to explicitly call out public static final String fields that are not configs.
*/
public @interface NotConf {
diff --git a/storm-client/src/jvm/org/apache/storm/validation/Validated.java b/storm-client/src/jvm/org/apache/storm/validation/Validated.java
index 3185e4d..b19af76 100644
--- a/storm-client/src/jvm/org/apache/storm/validation/Validated.java
+++ b/storm-client/src/jvm/org/apache/storm/validation/Validated.java
@@ -13,7 +13,7 @@
package org.apache.storm.validation;
/**
- * An interface that is used to inform config validation what to look at
+ * An interface that is used to inform config validation what to look at.
*/
public interface Validated {
//Empty
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
index bb22cb3..910b600 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java
@@ -67,10 +67,10 @@
@Override
public String toString() {
- return "CountEvictionPolicy{" +
- "threshold=" + threshold +
- ", currentCount=" + currentCount +
- '}';
+ return "CountEvictionPolicy{"
+ + "threshold=" + threshold
+ + ", currentCount=" + currentCount
+ + '}';
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/CountTriggerPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/CountTriggerPolicy.java
index 38b9025..096a02c 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/CountTriggerPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/CountTriggerPolicy.java
@@ -71,10 +71,10 @@
@Override
public String toString() {
- return "CountTriggerPolicy{" +
- "count=" + count +
- ", currentCount=" + currentCount +
- ", started=" + started +
- '}';
+ return "CountTriggerPolicy{"
+ + "count=" + count
+ + ", currentCount=" + currentCount
+ + ", started=" + started
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/EventImpl.java b/storm-client/src/jvm/org/apache/storm/windowing/EventImpl.java
index 71440fe..012e3b9 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/EventImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/EventImpl.java
@@ -38,9 +38,9 @@
@Override
public String toString() {
- return "EventImpl{" +
- "event=" + event +
- ", ts=" + ts +
- '}';
+ return "EventImpl{"
+ + "event=" + event
+ + ", ts=" + ts
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/EvictionContext.java b/storm-client/src/jvm/org/apache/storm/windowing/EvictionContext.java
index 872c0ea..1f600d6 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/EvictionContext.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/EvictionContext.java
@@ -13,7 +13,7 @@
package org.apache.storm.windowing;
/**
- * Context information that can be used by the eviction policy
+ * Context information that can be used by the eviction policy.
*/
public interface EvictionContext {
/**
@@ -25,7 +25,7 @@
Long getReferenceTime();
/**
- * Returns the sliding count for count based windows
+ * Returns the sliding count for count based windows.
*
* @return the sliding count
*/
@@ -33,7 +33,7 @@
/**
- * Returns the sliding interval for time based windows
+ * Returns the sliding interval for time based windows.
*
* @return the sliding interval
*/
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/StatefulWindowManager.java b/storm-client/src/jvm/org/apache/storm/windowing/StatefulWindowManager.java
index bf294ac..0222540 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/StatefulWindowManager.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/StatefulWindowManager.java
@@ -12,6 +12,10 @@
package org.apache.storm.windowing;
+import static org.apache.storm.windowing.EvictionPolicy.Action.EXPIRE;
+import static org.apache.storm.windowing.EvictionPolicy.Action.PROCESS;
+import static org.apache.storm.windowing.EvictionPolicy.Action.STOP;
+
import java.util.Collection;
import java.util.Iterator;
import java.util.NoSuchElementException;
@@ -19,10 +23,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.windowing.EvictionPolicy.Action.EXPIRE;
-import static org.apache.storm.windowing.EvictionPolicy.Action.PROCESS;
-import static org.apache.storm.windowing.EvictionPolicy.Action.STOP;
-
/**
* Window manager that handles windows with state persistence.
*/
@@ -34,7 +34,7 @@
}
/**
- * Constructs a {@link StatefulWindowManager}
+ * Constructs a {@link StatefulWindowManager}.
*
* @param lifecycleListener the {@link WindowLifecycleListener}
* @param queue a collection where the events in the window can be enqueued. <br/>
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
index 7dc62d1..8e15750 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java
@@ -26,7 +26,7 @@
private long delta;
/**
- * Constructs a TimeEvictionPolicy that evicts events older than the given window length in millis
+ * Constructs a TimeEvictionPolicy that evicts events older than the given window length in millis.
*
* @param windowLength the duration in milliseconds
*/
@@ -70,8 +70,8 @@
} else {
delta = context.getReferenceTime() - prevContext.getReferenceTime() - context.getSlidingInterval();
if (Math.abs(delta) > 100) {
- LOG.warn("Possible clock drift or long running computation in window; " +
- "Previous eviction time: {}, current eviction time: {}",
+ LOG.warn("Possible clock drift or long running computation in window; "
+ + "Previous eviction time: {}, current eviction time: {}",
prevContext.getReferenceTime(),
context.getReferenceTime());
}
@@ -96,9 +96,9 @@
@Override
public String toString() {
- return "TimeEvictionPolicy{" +
- "windowLength=" + windowLength +
- ", evictionContext=" + evictionContext +
- '}';
+ return "TimeEvictionPolicy{"
+ + "windowLength=" + windowLength
+ + ", evictionContext=" + evictionContext
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/TimeTriggerPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/TimeTriggerPolicy.java
index c58dafa..31f83c0 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/TimeTriggerPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/TimeTriggerPolicy.java
@@ -79,9 +79,9 @@
@Override
public String toString() {
- return "TimeTriggerPolicy{" +
- "duration=" + duration +
- '}';
+ return "TimeTriggerPolicy{"
+ + "duration=" + duration
+ + '}';
}
/*
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java b/storm-client/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
index 39afe4e..bbf8614 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java
@@ -65,11 +65,11 @@
@Override
public String toString() {
- return "TupleWindowImpl{" +
- "tuples=" + tuples +
- ", newTuples=" + newTuples +
- ", expiredTuples=" + expiredTuples +
- '}';
+ return "TupleWindowImpl{"
+ + "tuples=" + tuples
+ + ", newTuples=" + newTuples
+ + ", expiredTuples=" + expiredTuples
+ + '}';
}
@Override
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
index b76f605..ca6201a 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java
@@ -110,7 +110,6 @@
@Override
public String toString() {
- return "WatermarkCountEvictionPolicy{" +
- "} " + super.toString();
+ return "WatermarkCountEvictionPolicy{" + "} " + super.toString();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountTriggerPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountTriggerPolicy.java
index 669e7fd..3055187 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountTriggerPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkCountTriggerPolicy.java
@@ -85,10 +85,10 @@
@Override
public String toString() {
- return "WatermarkCountTriggerPolicy{" +
- "count=" + count +
- ", lastProcessedTs=" + lastProcessedTs +
- ", started=" + started +
- '}';
+ return "WatermarkCountTriggerPolicy{"
+ + "count=" + count
+ + ", lastProcessedTs=" + lastProcessedTs
+ + ", started=" + started
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
index 8865383..7638849 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java
@@ -67,9 +67,10 @@
@Override
public String toString() {
- return "WatermarkTimeEvictionPolicy{" +
- "lag=" + lag +
- "} " + super.toString();
+ return "WatermarkTimeEvictionPolicy{"
+ + "lag=" + lag
+ + "} "
+ + super.toString();
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeTriggerPolicy.java b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeTriggerPolicy.java
index 5695267..9758eb8 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeTriggerPolicy.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WatermarkTimeTriggerPolicy.java
@@ -118,10 +118,10 @@
@Override
public String toString() {
- return "WatermarkTimeTriggerPolicy{" +
- "slidingIntervalMs=" + slidingIntervalMs +
- ", nextWindowEndTs=" + nextWindowEndTs +
- ", started=" + started +
- '}';
+ return "WatermarkTimeTriggerPolicy{"
+ + "slidingIntervalMs=" + slidingIntervalMs
+ + ", nextWindowEndTs=" + nextWindowEndTs
+ + ", started=" + started
+ + '}';
}
}
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java b/storm-client/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
index 2adc2e0..ea9db39 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java
@@ -17,20 +17,20 @@
import java.util.function.Supplier;
/**
- * A callback for expiry, activation of events tracked by the {@link WindowManager}
+ * A callback for expiry, activation of events tracked by the {@link WindowManager}.
*
* @param <T> The type of Event in the window (e.g. Tuple).
*/
public interface WindowLifecycleListener<T> {
/**
- * Called on expiry of events from the window due to {@link EvictionPolicy}
+ * Called on expiry of events from the window due to {@link EvictionPolicy}.
*
* @param events the expired events
*/
void onExpiry(List<T> events);
/**
- * Called on activation of the window due to the {@link TriggerPolicy}
+ * Called on activation of the window due to the {@link TriggerPolicy}.
*
* @param events the list of current events in the window.
* @param newEvents the newly added events since last activation.
diff --git a/storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java b/storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java
index a9e6acc..9043020 100644
--- a/storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java
+++ b/storm-client/src/jvm/org/apache/storm/windowing/WindowManager.java
@@ -12,6 +12,10 @@
package org.apache.storm.windowing;
+import static org.apache.storm.windowing.EvictionPolicy.Action.EXPIRE;
+import static org.apache.storm.windowing.EvictionPolicy.Action.PROCESS;
+import static org.apache.storm.windowing.EvictionPolicy.Action.STOP;
+
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
@@ -28,10 +32,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.storm.windowing.EvictionPolicy.Action.EXPIRE;
-import static org.apache.storm.windowing.EvictionPolicy.Action.PROCESS;
-import static org.apache.storm.windowing.EvictionPolicy.Action.STOP;
-
/**
* Tracks a window of events and fires {@link WindowLifecycleListener} callbacks on expiry of events or activation of the window due to
* {@link TriggerPolicy}.
@@ -42,7 +42,7 @@
/**
* Expire old events every EXPIRE_EVENTS_THRESHOLD to keep the window size in check.
*
- * Note that if the eviction policy is based on watermarks, events will not be evicted until a new watermark would cause them to be
+ * <p>Note that if the eviction policy is based on watermarks, events will not be evicted until a new watermark would cause them to be
* considered expired anyway, regardless of this limit
*/
public static final int EXPIRE_EVENTS_THRESHOLD = 100;
@@ -63,7 +63,7 @@
}
/**
- * Constructs a {@link WindowManager}
+ * Constructs a {@link WindowManager}.
*
* @param lifecycleListener the {@link WindowLifecycleListener}
* @param queue a collection where the events in the window can be enqueued. <br/>
@@ -107,7 +107,7 @@
}
/**
- * Tracks a window event
+ * Tracks a window event.
*
* @param windowEvent the window event to track
*/
@@ -225,7 +225,7 @@
}
/**
- * Scans the event queue and returns the next earliest event ts between the startTs and endTs
+ * Scans the event queue and returns the next earliest event ts between the startTs and endTs.
*
* @param startTs the start ts (exclusive)
* @param endTs the end ts (inclusive)
@@ -285,10 +285,10 @@
@Override
public String toString() {
- return "WindowManager{" +
- "evictionPolicy=" + evictionPolicy +
- ", triggerPolicy=" + triggerPolicy +
- '}';
+ return "WindowManager{"
+ + "evictionPolicy=" + evictionPolicy
+ + ", triggerPolicy=" + triggerPolicy
+ + '}';
}
public void restoreState(Map<String, Optional<?>> state) {
diff --git a/storm-client/src/jvm/org/apache/storm/zookeeper/ClientZookeeper.java b/storm-client/src/jvm/org/apache/storm/zookeeper/ClientZookeeper.java
index 6cf0d33..0eeed37 100644
--- a/storm-client/src/jvm/org/apache/storm/zookeeper/ClientZookeeper.java
+++ b/storm-client/src/jvm/org/apache/storm/zookeeper/ClientZookeeper.java
@@ -234,7 +234,7 @@
}
/**
- * Get the data along with a version
+ * Get the data along with a version.
*
* @param zk the zk instance to use
* @param path the path to get it from
diff --git a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
index d335ccf..2ece764 100644
--- a/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
+++ b/storm-client/test/jvm/org/apache/storm/TestConfigValidate.java
@@ -48,13 +48,13 @@
import org.apache.storm.validation.ConfigValidation.StringValidator;
import org.apache.storm.validation.ConfigValidation.UserResourcePoolEntryValidator;
import org.apache.storm.validation.ConfigValidationAnnotations.NotNull;
-import org.apache.storm.validation.ConfigValidationAnnotations.isImplementationOfClass;
-import org.apache.storm.validation.ConfigValidationAnnotations.isListEntryCustom;
-import org.apache.storm.validation.ConfigValidationAnnotations.isListEntryType;
-import org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryCustom;
-import org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryType;
-import org.apache.storm.validation.ConfigValidationAnnotations.isNoDuplicateInList;
-import org.apache.storm.validation.ConfigValidationAnnotations.isString;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsImplementationOfClass;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsListEntryCustom;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsListEntryType;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryCustom;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryType;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsNoDuplicateInList;
+import org.apache.storm.validation.ConfigValidationAnnotations.IsString;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
@@ -766,34 +766,34 @@
}
public class TestConfig extends HashMap<String, Object> {
- @isMapEntryType(keyType = String.class, valueType = Integer.class)
+ @IsMapEntryType(keyType = String.class, valueType = Integer.class)
public static final String TEST_MAP_CONFIG = "test.map.config";
- @isMapEntryCustom(
+ @IsMapEntryCustom(
keyValidatorClasses = { StringValidator.class },
valueValidatorClasses = { PositiveNumberValidator.class, IntegerValidator.class })
public static final String TEST_MAP_CONFIG_2 = "test.map.config.2";
- @isListEntryType(type = Number.class)
+ @IsListEntryType(type = Number.class)
@NotNull
public static final String TEST_MAP_CONFIG_3 = "test.map.config.3";
- @isListEntryCustom(
+ @IsListEntryCustom(
entryValidatorClasses = { PositiveNumberValidator.class, NotNullValidator.class })
- @isNoDuplicateInList
+ @IsNoDuplicateInList
public static final String TEST_MAP_CONFIG_4 = "test.map.config.4";
- @isString(acceptedValues = { "aaa", "bbb", "ccc" })
+ @IsString(acceptedValues = { "aaa", "bbb", "ccc" })
public static final String TEST_MAP_CONFIG_5 = "test.map.config.5";
- @isMapEntryCustom(keyValidatorClasses = { StringValidator.class }, valueValidatorClasses = { ImpersonationAclUserEntryValidator
+ @IsMapEntryCustom(keyValidatorClasses = { StringValidator.class }, valueValidatorClasses = { ImpersonationAclUserEntryValidator
.class })
public static final String TEST_MAP_CONFIG_6 = "test.map.config.6";
- @isMapEntryCustom(keyValidatorClasses = { StringValidator.class }, valueValidatorClasses = { UserResourcePoolEntryValidator.class })
+ @IsMapEntryCustom(keyValidatorClasses = { StringValidator.class }, valueValidatorClasses = { UserResourcePoolEntryValidator.class })
public static final String TEST_MAP_CONFIG_7 = "test.map.config.7";
- @isImplementationOfClass(implementsClass = org.apache.storm.networktopography.DNSToSwitchMapping.class)
+ @IsImplementationOfClass(implementsClass = org.apache.storm.networktopography.DNSToSwitchMapping.class)
@NotNull
public static final String TEST_MAP_CONFIG_8 = "test.map.config.8";
}
diff --git a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
index bbd59c2..26cacff 100644
--- a/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
+++ b/storm-server/src/main/java/org/apache/storm/DaemonConfig.java
@@ -32,18 +32,18 @@
import org.apache.storm.validation.Validated;
import static org.apache.storm.validation.ConfigValidationAnnotations.NotNull;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isBoolean;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isImplementationOfClass;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isInteger;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isListEntryCustom;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryCustom;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isMapEntryType;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isNoDuplicateInList;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isNumber;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isPositiveNumber;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isString;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isStringList;
-import static org.apache.storm.validation.ConfigValidationAnnotations.isStringOrStringList;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsBoolean;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsImplementationOfClass;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsInteger;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsListEntryCustom;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryCustom;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsMapEntryType;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsNoDuplicateInList;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsNumber;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsPositiveNumber;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsString;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsStringList;
+import static org.apache.storm.validation.ConfigValidationAnnotations.IsStringOrStringList;
import static org.apache.storm.validation.ConfigValidationAnnotations.Password;
/**
@@ -58,33 +58,33 @@
/**
* We check with this interval that whether the Netty channel is writable and try to write pending messages.
*/
- @isInteger
+ @IsInteger
public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms";
/**
* A list of daemon metrics reporter plugin class names. These plugins must implement {@link
* org.apache.storm.daemon.metrics.reporters.PreparableReporter} interface.
*/
- @isStringList
+ @IsStringList
public static final String STORM_DAEMON_METRICS_REPORTER_PLUGINS = "storm.daemon.metrics.reporter.plugins";
/**
* A specify domain for daemon metrics reporter plugin to limit reporting to specific domain.
*/
- @isString
+ @IsString
public static final String STORM_DAEMON_METRICS_REPORTER_PLUGIN_DOMAIN = "storm.daemon.metrics.reporter.plugin.domain";
/**
* A specify csv reporter directory for CvsPreparableReporter daemon metrics reporter.
*/
- @isString
+ @IsString
public static final String STORM_DAEMON_METRICS_REPORTER_CSV_LOG_DIR = "storm.daemon.metrics.reporter.csv.log.dir";
/**
* A directory that holds configuration files for log4j2. It can be either a relative or an absolute directory. If relative, it is
* relative to the storm's home directory.
*/
- @isString
+ @IsString
public static final String STORM_LOG4J2_CONF_DIR = "storm.log4j2.conf.dir";
/**
@@ -92,39 +92,39 @@
*
* If this is not set, a default system scheduler will be used.
*/
- @isString
+ @IsString
public static final String STORM_SCHEDULER = "storm.scheduler";
/**
* Max time to attempt to schedule one topology. The default is 60 seconds
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String SCHEDULING_TIMEOUT_SECONDS_PER_TOPOLOGY = "scheduling.timeout.seconds.per.topology";
/**
* The number of seconds that the blacklist scheduler will concern of bad slots or supervisors.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String BLACKLIST_SCHEDULER_TOLERANCE_TIME = "blacklist.scheduler.tolerance.time.secs";
/**
* The number of hit count that will trigger blacklist in tolerance time.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String BLACKLIST_SCHEDULER_TOLERANCE_COUNT = "blacklist.scheduler.tolerance.count";
/**
* The number of seconds that the blacklisted slots or supervisor will be resumed.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String BLACKLIST_SCHEDULER_RESUME_TIME = "blacklist.scheduler.resume.time.secs";
/**
* The class that the blacklist scheduler will report the blacklist.
*/
@NotNull
- @isImplementationOfClass(implementsClass = IReporter.class)
+ @IsImplementationOfClass(implementsClass = IReporter.class)
public static final String BLACKLIST_SCHEDULER_REPORTER = "blacklist.scheduler.reporter";
/**
@@ -134,7 +134,7 @@
* get odd behavior when the cluster is full and there are blacklisted nodes.
*/
@NotNull
- @isImplementationOfClass(implementsClass = IBlacklistStrategy.class)
+ @IsImplementationOfClass(implementsClass = IBlacklistStrategy.class)
public static final String BLACKLIST_SCHEDULER_STRATEGY = "blacklist.scheduler.strategy";
/**
@@ -143,40 +143,40 @@
* <p/>
* If this is not set, we will not display resource capacity and usage on the UI.
*/
- @isBoolean
+ @IsBoolean
public static final String SCHEDULER_DISPLAY_RESOURCE = "scheduler.display.resource";
/**
* The directory where storm's health scripts go.
*/
- @isString
+ @IsString
public static final String STORM_HEALTH_CHECK_DIR = "storm.health.check.dir";
/**
* The time to allow any given healthcheck script to run before it is marked failed due to timeout.
*/
- @isNumber
+ @IsNumber
public static final String STORM_HEALTH_CHECK_TIMEOUT_MS = "storm.health.check.timeout.ms";
/**
* This is the user that the Nimbus daemon process is running as. May be used when security is enabled to authorize actions in the
* cluster.
*/
- @isString
+ @IsString
public static final String NIMBUS_DAEMON_USER = "nimbus.daemon.user";
/**
* This parameter is used by the storm-deploy project to configure the jvm options for the nimbus daemon.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String NIMBUS_CHILDOPTS = "nimbus.childopts";
/**
* How long without heartbeating a task can go before nimbus will consider the task dead and reassign it to another location.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs";
@@ -184,8 +184,8 @@
* How often nimbus should wake up to check heartbeats and do reassignments. Note that if a machine ever goes down Nimbus will
* immediately wake up and take action. This parameter is for checking for failures when there's no explicit event like that occurring.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs";
/**
@@ -193,8 +193,8 @@
*
* @see #NIMBUS_INBOX_JAR_EXPIRATION_SECS
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_CLEANUP_INBOX_FREQ_SECS = "nimbus.cleanup.inbox.freq.secs";
/**
@@ -206,14 +206,14 @@
*
* @see #NIMBUS_CLEANUP_INBOX_FREQ_SECS
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_INBOX_JAR_EXPIRATION_SECS = "nimbus.inbox.jar.expiration.secs";
/**
* How long before a supervisor can go without heartbeating before nimbus considers it dead and stops assigning new work to it.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs";
/**
@@ -223,192 +223,192 @@
* <p>A separate timeout exists for launch because there can be quite a bit of overhead
* to launching new JVM's and configuring them.</p>
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs";
/**
* During upload/download with the master, how long an upload or download connection is idle before nimbus considers it dead and drops
* the connection.
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs";
/**
* A custom class that implements ITopologyValidator that is run whenever a topology is submitted. Can be used to provide
* business-specific logic for whether topologies are allowed to run or not.
*/
- @isString
+ @IsString
public static final String NIMBUS_TOPOLOGY_VALIDATOR = "nimbus.topology.validator";
/**
* Class name for authorization plugin for Nimbus.
*/
- @isString
+ @IsString
public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer";
/**
* Class name for authorization plugin for supervisor.
*/
- @isImplementationOfClass(implementsClass = IAuthorizer.class)
- @isString
+ @IsImplementationOfClass(implementsClass = IAuthorizer.class)
+ @IsString
public static final String SUPERVISOR_AUTHORIZER = "supervisor.authorizer";
/**
* Impersonation user ACL config entries.
*/
- @isString
+ @IsString
public static final String NIMBUS_IMPERSONATION_AUTHORIZER = "nimbus.impersonation.authorizer";
/**
* How often nimbus should wake up to renew credentials if needed.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs";
/**
* FQCN of a class that implements {@code I} @see org.apache.storm.nimbus.ITopologyActionNotifierPlugin for details.
*/
- @isImplementationOfClass(implementsClass = ITopologyActionNotifierPlugin.class)
+ @IsImplementationOfClass(implementsClass = ITopologyActionNotifierPlugin.class)
public static final String NIMBUS_TOPOLOGY_ACTION_NOTIFIER_PLUGIN = "nimbus.topology.action.notifier.plugin.class";
/**
* This controls the number of working threads for distributing master assignments to supervisors.
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_ASSIGNMENTS_SERVICE_THREADS = "nimbus.assignments.service.threads";
/**
* This controls the number of working thread queue size of assignment service.
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_ASSIGNMENTS_SERVICE_THREAD_QUEUE_SIZE = "nimbus.assignments.service.thread.queue.size";
/**
* class controls heartbeats recovery strategy.
*/
- @isString
+ @IsString
public static final String NIMBUS_WORKER_HEARTBEATS_RECOVERY_STRATEGY_CLASS = "nimbus.worker.heartbeats.recovery.strategy.class";
/**
* This controls the number of milliseconds nimbus will wait before deleting a topology blobstore once detected it is able to delete.
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_TOPOLOGY_BLOBSTORE_DELETION_DELAY_MS = "nimbus.topology.blobstore.deletion.delay.ms";
/**
* Storm UI binds to this host/interface.
*/
- @isString
+ @IsString
public static final String UI_HOST = "ui.host";
/**
* Storm UI binds to this port.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String UI_PORT = "ui.port";
/**
* This controls wheather Storm UI should bind to http port even if ui.port is > 0.
*/
- @isBoolean
+ @IsBoolean
public static final String UI_DISABLE_HTTP_BINDING = "ui.disable.http.binding";
/**
* This controls whether Storm UI displays spout lag for the Kafka spout.
*/
- @isBoolean
+ @IsBoolean
public static final String UI_DISABLE_SPOUT_LAG_MONITORING = "ui.disable.spout.lag.monitoring";
/**
* This controls wheather Storm Logviewer should bind to http port even if logviewer.port is > 0.
*/
- @isBoolean
+ @IsBoolean
public static final String LOGVIEWER_DISABLE_HTTP_BINDING = "logviewer.disable.http.binding";
/**
* This controls wheather Storm DRPC should bind to http port even if drpc.http.port is > 0.
*/
- @isBoolean
+ @IsBoolean
public static final String DRPC_DISABLE_HTTP_BINDING = "drpc.disable.http.binding";
/**
* Storm UI Project BUGTRACKER Link for reporting issue.
*/
- @isString
+ @IsString
public static final String UI_PROJECT_BUGTRACKER_URL = "ui.project.bugtracker.url";
/**
* Storm UI Central Logging URL.
*/
- @isString
+ @IsString
public static final String UI_CENTRAL_LOGGING_URL = "ui.central.logging.url";
/**
* Storm UI drop-down pagination value. Set ui.pagination to be a positive integer or -1 (displays all entries). Valid values: -1, 10,
* 20, 25 etc.
*/
- @isInteger
+ @IsInteger
public static final String UI_PAGINATION = "ui.pagination";
/**
* HTTP UI port for log viewer.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String LOGVIEWER_PORT = "logviewer.port";
/**
* Childopts for log viewer java process.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts";
/**
* How often to clean up old log files.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String LOGVIEWER_CLEANUP_INTERVAL_SECS = "logviewer.cleanup.interval.secs";
/**
* How many minutes since a log was last modified for the log to be considered for clean-up.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String LOGVIEWER_CLEANUP_AGE_MINS = "logviewer.cleanup.age.mins";
/**
* The maximum number of bytes all worker log files can take up in MB.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String LOGVIEWER_MAX_SUM_WORKER_LOGS_SIZE_MB = "logviewer.max.sum.worker.logs.size.mb";
/**
* The maximum number of bytes per worker's files can take up in MB.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static final String LOGVIEWER_MAX_PER_WORKER_LOGS_SIZE_MB = "logviewer.max.per.worker.logs.size.mb";
/**
* Storm Logviewer HTTPS port. Logviewer must use HTTPS if Storm UI is using HTTPS.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String LOGVIEWER_HTTPS_PORT = "logviewer.https.port";
/**
* Path to the keystore containing the certs used by Storm Logviewer for HTTPS communications.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_HTTPS_KEYSTORE_PATH = "logviewer.https.keystore.path";
/**
* Password for the keystore for HTTPS for Storm Logviewer.
*/
- @isString
+ @IsString
@Password
public static final String LOGVIEWER_HTTPS_KEYSTORE_PASSWORD = "logviewer.https.keystore.password";
@@ -416,26 +416,26 @@
* Type of the keystore for HTTPS for Storm Logviewer. see http://docs.oracle.com/javase/8/docs/api/java/security/KeyStore.html for more
* details.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_HTTPS_KEYSTORE_TYPE = "logviewer.https.keystore.type";
/**
* Password to the private key in the keystore for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String LOGVIEWER_HTTPS_KEY_PASSWORD = "logviewer.https.key.password";
/**
* Path to the truststore containing the certs used by Storm Logviewer for HTTPS communications.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_HTTPS_TRUSTSTORE_PATH = "logviewer.https.truststore.path";
/**
* Password for the truststore for HTTPS for Storm Logviewer.
*/
- @isString
+ @IsString
@Password
public static final String LOGVIEWER_HTTPS_TRUSTSTORE_PASSWORD = "logviewer.https.truststore.password";
@@ -443,90 +443,90 @@
* Type of the truststore for HTTPS for Storm Logviewer. see http://docs.oracle.com/javase/8/docs/api/java/security/Truststore.html for
* more details.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_HTTPS_TRUSTSTORE_TYPE = "logviewer.https.truststore.type";
/**
* Password to the truststore used by Storm Logviewer setting up HTTPS (SSL).
*/
- @isBoolean
+ @IsBoolean
public static final String LOGVIEWER_HTTPS_WANT_CLIENT_AUTH = "logviewer.https.want.client.auth";
- @isBoolean
+ @IsBoolean
public static final String LOGVIEWER_HTTPS_NEED_CLIENT_AUTH = "logviewer.https.need.client.auth";
/**
* A list of users allowed to view logs via the Log Viewer.
*/
- @isStringList
+ @IsStringList
public static final String LOGS_USERS = "logs.users";
/**
* A list of groups allowed to view logs via the Log Viewer.
*/
- @isStringList
+ @IsStringList
public static final String LOGS_GROUPS = "logs.groups";
/**
* Appender name used by log viewer to determine log directory.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name";
/**
* A class implementing javax.servlet.Filter for authenticating/filtering Logviewer requests.
*/
- @isString
+ @IsString
public static final String LOGVIEWER_FILTER = "logviewer.filter";
/**
* Initialization parameters for the javax.servlet.Filter for Logviewer.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String LOGVIEWER_FILTER_PARAMS = "logviewer.filter.params";
/**
* Childopts for Storm UI Java process.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String UI_CHILDOPTS = "ui.childopts";
/**
* A class implementing javax.servlet.Filter for authenticating/filtering UI requests.
*/
- @isString
+ @IsString
public static final String UI_FILTER = "ui.filter";
/**
* Initialization parameters for the javax.servlet.Filter for UI.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String UI_FILTER_PARAMS = "ui.filter.params";
/**
* The size of the header buffer for the UI in bytes.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String UI_HEADER_BUFFER_BYTES = "ui.header.buffer.bytes";
/**
* This port is used by Storm UI for receiving HTTPS (SSL) requests from clients.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String UI_HTTPS_PORT = "ui.https.port";
/**
* Path to the keystore used by Storm UI for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
public static final String UI_HTTPS_KEYSTORE_PATH = "ui.https.keystore.path";
/**
* Password to the keystore used by Storm UI for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String UI_HTTPS_KEYSTORE_PASSWORD = "ui.https.keystore.password";
@@ -535,26 +535,26 @@
* .html
* for more details.
*/
- @isString
+ @IsString
public static final String UI_HTTPS_KEYSTORE_TYPE = "ui.https.keystore.type";
/**
* Password to the private key in the keystore for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String UI_HTTPS_KEY_PASSWORD = "ui.https.key.password";
/**
* Path to the truststore used by Storm UI setting up HTTPS (SSL).
*/
- @isString
+ @IsString
public static final String UI_HTTPS_TRUSTSTORE_PATH = "ui.https.truststore.path";
/**
* Password to the truststore used by Storm UI setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String UI_HTTPS_TRUSTSTORE_PASSWORD = "ui.https.truststore.password";
@@ -563,55 +563,55 @@
* .com/javase/7/docs/api/java/security/KeyStore.html
* for more details.
*/
- @isString
+ @IsString
public static final String UI_HTTPS_TRUSTSTORE_TYPE = "ui.https.truststore.type";
/**
* Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
*/
- @isBoolean
+ @IsBoolean
public static final String UI_HTTPS_WANT_CLIENT_AUTH = "ui.https.want.client.auth";
- @isBoolean
+ @IsBoolean
public static final String UI_HTTPS_NEED_CLIENT_AUTH = "ui.https.need.client.auth";
/**
* The maximum number of threads that should be used by the Pacemaker. When Pacemaker gets loaded it will spawn new threads, up to this
* many total, to handle the load.
*/
- @isNumber
- @isPositiveNumber
+ @IsNumber
+ @IsPositiveNumber
public static final String PACEMAKER_MAX_THREADS = "pacemaker.max.threads";
/**
* This parameter is used by the storm-deploy project to configure the jvm options for the pacemaker daemon.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String PACEMAKER_CHILDOPTS = "pacemaker.childopts";
/**
* This port is used by Storm DRPC for receiving HTTP DPRC requests from clients.
*/
- @isInteger
+ @IsInteger
public static final String DRPC_HTTP_PORT = "drpc.http.port";
/**
* This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
*/
- @isInteger
+ @IsInteger
public static final String DRPC_HTTPS_PORT = "drpc.https.port";
/**
* Path to the keystore used by Storm DRPC for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path";
/**
* Password to the keystore used by Storm DRPC for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password";
@@ -620,26 +620,26 @@
* .com/javase/7/docs/api/java/security/KeyStore.html
* for more details.
*/
- @isString
+ @IsString
public static final String DRPC_HTTPS_KEYSTORE_TYPE = "drpc.https.keystore.type";
/**
* Password to the private key in the keystore for setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String DRPC_HTTPS_KEY_PASSWORD = "drpc.https.key.password";
/**
* Path to the truststore used by Storm DRPC setting up HTTPS (SSL).
*/
- @isString
+ @IsString
public static final String DRPC_HTTPS_TRUSTSTORE_PATH = "drpc.https.truststore.path";
/**
* Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
*/
- @isString
+ @IsString
@Password
public static final String DRPC_HTTPS_TRUSTSTORE_PASSWORD = "drpc.https.truststore.password";
@@ -648,22 +648,22 @@
* .com/javase/7/docs/api/java/security/KeyStore.html
* for more details.
*/
- @isString
+ @IsString
public static final String DRPC_HTTPS_TRUSTSTORE_TYPE = "drpc.https.truststore.type";
/**
* Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
*/
- @isBoolean
+ @IsBoolean
public static final String DRPC_HTTPS_WANT_CLIENT_AUTH = "drpc.https.want.client.auth";
- @isBoolean
+ @IsBoolean
public static final String DRPC_HTTPS_NEED_CLIENT_AUTH = "drpc.https.need.client.auth";
/**
* Class name for authorization plugin for DRPC client.
*/
- @isString
+ @IsString
public static final String DRPC_AUTHORIZER = "drpc.authorizer";
/**
@@ -671,111 +671,111 @@
* timeout on the DRPC client, and separately based on the topology message timeout for the topology implementing the DRPC function.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String DRPC_REQUEST_TIMEOUT_SECS = "drpc.request.timeout.secs";
/**
* Childopts for Storm DRPC Java process.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String DRPC_CHILDOPTS = "drpc.childopts";
/**
* the metadata configured on the supervisor.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta";
/**
* A list of ports that can run workers on this supervisor. Each worker uses one port, and the supervisor will only run one worker per
* port. Use this configuration to tune how many workers run on each machine.
*/
- @isNoDuplicateInList
+ @IsNoDuplicateInList
@NotNull
- @isListEntryCustom(entryValidatorClasses = { ConfigValidation.IntegerValidator.class, ConfigValidation.PositiveNumberValidator.class })
+ @IsListEntryCustom(entryValidatorClasses = { ConfigValidation.IntegerValidator.class, ConfigValidation.PositiveNumberValidator.class })
public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
/**
* What blobstore implementation the supervisor should use.
*/
- @isString
+ @IsString
public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
/**
* The distributed cache target size in MB. This is a soft limit to the size of the distributed cache contents.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
/**
* The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup anything over the cache target size.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
/**
* What blobstore download parallelism the supervisor should use.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
/**
* Maximum number of retries a supervisor is allowed to make for downloading a blob.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
/**
* What blobstore implementation nimbus should use.
*/
- @isString
+ @IsString
public static final String NIMBUS_BLOBSTORE = "nimbus.blobstore.class";
/**
* During operations with the blob store, via master, how long a connection is idle before nimbus considers it dead and drops the
* session and any associated connections.
*/
- @isPositiveNumber
- @isInteger
+ @IsPositiveNumber
+ @IsInteger
public static final String NIMBUS_BLOBSTORE_EXPIRATION_SECS = "nimbus.blobstore.expiration.secs";
/**
* A number representing the maximum number of workers any single topology can acquire.
* This will be ignored if the Resource Aware Scheduler is used.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology";
/**
* A class implementing javax.servlet.Filter for DRPC HTTP requests.
*/
- @isString
+ @IsString
public static final String DRPC_HTTP_FILTER = "drpc.http.filter";
/**
* Initialization parameters for the javax.servlet.Filter of the DRPC HTTP service.
*/
- @isMapEntryType(keyType = String.class, valueType = String.class)
+ @IsMapEntryType(keyType = String.class, valueType = String.class)
public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params";
/**
* A number representing the maximum number of executors any single topology can acquire.
*/
- @isInteger
- @isPositiveNumber(includeZero = true)
+ @IsInteger
+ @IsPositiveNumber(includeZero = true)
public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology";
/**
* This parameter is used by the storm-deploy project to configure the jvm options for the supervisor daemon.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts";
/**
@@ -783,8 +783,8 @@
* This value override supervisor.worker.timeout.secs during launch because there is additional overhead to starting and configuring the
* JVM on launch.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
@NotNull
public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs";
@@ -792,42 +792,42 @@
* Whether or not the supervisor should launch workers assigned to it. Defaults to true -- and you should probably never change this
* value. This configuration is used in the Storm unit tests.
*/
- @isBoolean
+ @IsBoolean
public static final String SUPERVISOR_ENABLE = "supervisor.enable";
/**
* how often the supervisor sends a heartbeat to the master.
*/
- @isInteger
+ @IsInteger
public static final String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs";
/**
* How often the supervisor checks the worker heartbeats to see if any of them need to be restarted.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs";
/**
* The jvm profiler opts provided to workers launched by this supervisor.
*/
- @isStringOrStringList
+ @IsStringOrStringList
public static final String WORKER_PROFILER_CHILDOPTS = "worker.profiler.childopts";
/**
* Enable profiling of worker JVMs using Oracle's Java Flight Recorder. Unlocking commercial features requires a special license from
* Oracle. See http://www.oracle.com/technetwork/java/javase/terms/products/index.html
*/
- @isBoolean
+ @IsBoolean
public static final String WORKER_PROFILER_ENABLED = "worker.profiler.enabled";
/**
* The command launched supervisor with worker arguments pid, action and [target_directory] Where action is - start profile, stop
* profile, jstack, heapdump and kill against pid.
*/
- @isString
+ @IsString
public static final String WORKER_PROFILER_COMMAND = "worker.profiler.command";
/**
@@ -835,21 +835,21 @@
* routed cluster related metrics data. Each listed class maps 1:1 to a ClusterMetricsConsumerExecutor and they're executed in Nimbus.
* Only consumers which run in leader Nimbus receives metrics data.
*/
- @isListEntryCustom(entryValidatorClasses = { ConfigValidation.ClusterMetricRegistryValidator.class })
+ @IsListEntryCustom(entryValidatorClasses = { ConfigValidation.ClusterMetricRegistryValidator.class })
public static final String STORM_CLUSTER_METRICS_CONSUMER_REGISTER = "storm.cluster.metrics.consumer.register";
/**
* How often cluster metrics data is published to metrics consumer.
*/
@NotNull
- @isPositiveNumber
+ @IsPositiveNumber
public static final String STORM_CLUSTER_METRICS_CONSUMER_PUBLISH_INTERVAL_SECS =
"storm.cluster.metrics.consumer.publish.interval.secs";
/**
* Enables user-first classpath. See topology.classpath.beginning.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_TOPOLOGY_CLASSPATH_BEGINNING_ENABLED = "storm.topology.classpath.beginning.enabled";
/**
@@ -857,21 +857,21 @@
* the JVM where to look for native libraries. It is necessary to set this config correctly since Storm uses the ZeroMQ and JZMQ native
* libs.
*/
- @isString
+ @IsString
public static final String JAVA_LIBRARY_PATH = "java.library.path";
/**
* The path to use as the zookeeper dir when running a zookeeper server via "storm dev-zookeeper". This zookeeper instance is only
* intended for development; it is not a production grade zookeeper setup.
*/
- @isString
+ @IsString
public static final String DEV_ZOOKEEPER_PATH = "dev.zookeeper.path";
/**
* A map from topology name to the number of machines that should be dedicated for that topology. Set storm.scheduler to
* org.apache.storm.scheduler.IsolationScheduler to make use of the isolation scheduler.
*/
- @isMapEntryType(keyType = String.class, valueType = Number.class)
+ @IsMapEntryType(keyType = String.class, valueType = Number.class)
public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines";
/**
@@ -879,44 +879,44 @@
* directory, the file with the largest lexographic name will be returned. Users need to add "artifactory+" to the beginning of the real
* URI to use ArtifactoryConfigLoader. For FileConfigLoader, this is the URI pointing to a file.
*/
- @isString
+ @IsString
public static final String SCHEDULER_CONFIG_LOADER_URI = "scheduler.config.loader.uri";
/**
* It is the frequency at which the plugin will call out to artifactory instead of returning the most recently cached result. Currently
* it's only used in ArtifactoryConfigLoader.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String SCHEDULER_CONFIG_LOADER_POLLTIME_SECS = "scheduler.config.loader.polltime.secs";
/**
* It is the amount of time an http connection to the artifactory server will wait before timing out. Currently it's only used in
* ArtifactoryConfigLoader.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String SCHEDULER_CONFIG_LOADER_TIMEOUT_SECS = "scheduler.config.loader.timeout.secs";
/**
* It is the part of the uri, configurable in Artifactory, which represents the top of the directory tree. It's only used in
* ArtifactoryConfigLoader.
*/
- @isString
+ @IsString
public static final String SCHEDULER_CONFIG_LOADER_ARTIFACTORY_BASE_DIRECTORY = "scheduler.config.loader.artifactory.base.directory";
/**
* A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler to
* org.apache.storm.scheduler.multitenant.MultitenantScheduler
*/
- @isMapEntryType(keyType = String.class, valueType = Number.class)
+ @IsMapEntryType(keyType = String.class, valueType = Number.class)
public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools";
/**
* A map of users to another map of the resource guarantees of the user. Used by Resource Aware Scheduler to ensure per user resource
* guarantees.
*/
- @isMapEntryCustom(
+ @IsMapEntryCustom(
keyValidatorClasses = { ConfigValidation.StringValidator.class },
valueValidatorClasses = { ConfigValidation.UserResourcePoolEntryValidator.class })
public static final String RESOURCE_AWARE_SCHEDULER_USER_POOLS = "resource.aware.scheduler.user.pools";
@@ -925,34 +925,34 @@
* the class that specifies the scheduling priority strategy to use in ResourceAwareScheduler.
*/
@NotNull
- @isImplementationOfClass(implementsClass = ISchedulingPriorityStrategy.class)
+ @IsImplementationOfClass(implementsClass = ISchedulingPriorityStrategy.class)
public static final String RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY = "resource.aware.scheduler.priority.strategy";
/**
* The maximum number of times that the RAS will attempt to schedule a topology. The default is 5.
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String RESOURCE_AWARE_SCHEDULER_MAX_TOPOLOGY_SCHEDULING_ATTEMPTS =
"resource.aware.scheduler.max.topology.scheduling.attempts";
/*
* The maximum number of states that will be searched looking for a solution in the constraint solver strategy
*/
- @isInteger
- @isPositiveNumber
+ @IsInteger
+ @IsPositiveNumber
public static final String RESOURCE_AWARE_SCHEDULER_MAX_STATE_SEARCH = "resource.aware.scheduler.constraint.max.state.search";
/**
* How often nimbus's background thread to sync code for missing topologies should run.
*/
- @isInteger
+ @IsInteger
public static final String NIMBUS_CODE_SYNC_FREQ_SECS = "nimbus.code.sync.freq.secs";
/**
* The plugin to be used for resource isolation.
*/
- @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
+ @IsImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
public static final String STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
/**
@@ -962,26 +962,26 @@
/**
* resources to to be controlled by cgroups.
*/
- @isStringList
+ @IsStringList
public static final String STORM_CGROUP_RESOURCES = "storm.cgroup.resources";
/**
* name for the cgroup hierarchy.
*/
- @isString
+ @IsString
public static final String STORM_CGROUP_HIERARCHY_NAME = "storm.cgroup.hierarchy.name";
/**
* flag to determine whether to use a resource isolation plugin Also determines whether the unit tests for cgroup runs. If
* storm.resource.isolation.plugin.enable is set to false the unit tests for cgroups will not run
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE = "storm.resource.isolation.plugin.enable";
/**
* Class implementing MetricStore. Runs on Nimbus.
*/
@NotNull
- @isString
+ @IsString
// Validating class implementation could fail on non-Nimbus Daemons. Nimbus will catch the class not found on startup
// and log an error message, so just validating this as a String for now.
public static final String STORM_METRIC_STORE_CLASS = "storm.metricstore.class";
@@ -989,31 +989,31 @@
* Class implementing WorkerMetricsProcessor. Runs on Supervisors.
*/
@NotNull
- @isString
+ @IsString
public static final String STORM_METRIC_PROCESSOR_CLASS = "storm.metricprocessor.class";
/**
* RocksDB file location. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore implementation for the
* storm.metricstore.class.
*/
- @isString
+ @IsString
public static final String STORM_ROCKSDB_LOCATION = "storm.metricstore.rocksdb.location";
/**
* RocksDB create if missing flag. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore implementation for
* the storm.metricstore.class.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_ROCKSDB_CREATE_IF_MISSING = "storm.metricstore.rocksdb.create_if_missing";
/**
* RocksDB metadata cache capacity. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore implementation for
* the storm.metricstore.class.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ROCKSDB_METADATA_STRING_CACHE_CAPACITY = "storm.metricstore.rocksdb.metadata_string_cache_capacity";
/**
* RocksDB setting for length of metric retention. This setting is specific to the org.apache.storm.metricstore.rocksdb.RocksDbStore
* implementation for the storm.metricstore.class.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ROCKSDB_METRIC_RETENTION_HOURS = "storm.metricstore.rocksdb.retention_hours";
// Configs for memory enforcement done by the supervisor (not cgroups directly)
@@ -1022,62 +1022,62 @@
* .RocksDbStore
* implementation for the storm.metricstore.class.
*/
- @isInteger
+ @IsInteger
public static final String STORM_ROCKSDB_METRIC_DELETION_PERIOD_HOURS = "storm.metricstore.rocksdb.deletion_period_hours";
/**
* In nimbus on startup check if all of the zookeeper ACLs are correct before starting. If not don't start nimbus.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_NIMBUS_ZOOKEEPER_ACLS_CHECK = "storm.nimbus.zookeeper.acls.check";
/**
* In nimbus on startup check if all of the zookeeper ACLs are correct before starting. If not do your best to fix them before nimbus
* starts, if it cannot fix them nimbus will not start. This overrides any value set for storm.nimbus.zookeeper.acls.check.
*/
- @isBoolean
+ @IsBoolean
public static final String STORM_NIMBUS_ZOOKEEPER_ACLS_FIXUP = "storm.nimbus.zookeeper.acls.fixup";
/**
* Server side validation that @{see Config#TOPOLOGY_SCHEDULER_STRATEGY} is set ot a subclass of IStrategy.
*/
- @isImplementationOfClass(implementsClass = IStrategy.class)
+ @IsImplementationOfClass(implementsClass = IStrategy.class)
public static final String VALIDATE_TOPOLOGY_SCHEDULER_STRATEGY = Config.TOPOLOGY_SCHEDULER_STRATEGY;
/**
* Class name of the HTTP credentials plugin for the UI.
*/
- @isImplementationOfClass(implementsClass = IHttpCredentialsPlugin.class)
+ @IsImplementationOfClass(implementsClass = IHttpCredentialsPlugin.class)
public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin";
/**
* Class name of the HTTP credentials plugin for DRPC.
*/
- @isImplementationOfClass(implementsClass = IHttpCredentialsPlugin.class)
+ @IsImplementationOfClass(implementsClass = IHttpCredentialsPlugin.class)
public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin";
/**
* root directory for cgoups.
*/
- @isString
+ @IsString
public static String STORM_SUPERVISOR_CGROUP_ROOTDIR = "storm.supervisor.cgroup.rootdir";
/**
* the manually set memory limit (in MB) for each CGroup on supervisor node.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_WORKER_CGROUP_MEMORY_MB_LIMIT = "storm.worker.cgroup.memory.mb.limit";
/**
* the manually set cpu share for each CGroup on supervisor node.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_WORKER_CGROUP_CPU_LIMIT = "storm.worker.cgroup.cpu.limit";
/**
* full path to cgexec command.
*/
- @isString
+ @IsString
public static String STORM_CGROUP_CGEXEC_CMD = "storm.cgroup.cgexec.cmd";
/**
* Please use STORM_SUPERVISOR_MEMORY_LIMIT_TOLERANCE_MARGIN_MB instead. The amount of memory a worker can exceed its allocation before
* cgroup will kill it.
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static String STORM_CGROUP_MEMORY_LIMIT_TOLERANCE_MARGIN_MB =
"storm.cgroup.memory.limit.tolerance.margin.mb";
/**
@@ -1086,20 +1086,20 @@
* being able to launch processes in that cgroup. The common use case for this config is when the linux distribution
* that is used does not support the cgroup.clone_children config.
*/
- @isBoolean
+ @IsBoolean
public static String STORM_CGROUP_INHERIT_CPUSET_CONFIGS = "storm.cgroup.inherit.cpuset.configs";
/**
* Java does not always play nicely with cgroups. It is coming but not fully implemented and not for the way storm uses cgroups. In the
* short term you can disable the hard memory enforcement by cgroups and let the supervisor handle shooting workers going over their
* limit in a kinder way.
*/
- @isBoolean
+ @IsBoolean
public static String STORM_CGROUP_MEMORY_ENFORCEMENT_ENABLE = "storm.cgroup.memory.enforcement.enable";
/**
* Memory given to each worker for free (because java and storm have some overhead). This is memory on the box that the workers can use.
* This should not be included in SUPERVISOR_MEMORY_CAPACITY_MB, as nimbus does not use this memory for scheduling.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_SUPERVISOR_MEMORY_LIMIT_TOLERANCE_MARGIN_MB =
"storm.supervisor.memory.limit.tolerance.margin.mb";
/**
@@ -1108,7 +1108,7 @@
* STORM_SUPERVISOR_HARD_MEMORY_LIMIT_OVERAGE and which ever is greater is used for enforcement. This allows small workers to not be
* shot.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_SUPERVISOR_HARD_MEMORY_LIMIT_MULTIPLIER =
"storm.supervisor.hard.memory.limit.multiplier";
/**
@@ -1116,26 +1116,26 @@
* STORM_SUPERVISOR_HARD_LIMIT_MEMORY_MULTIPLIER and which ever is greater is used for enforcement. This allows small workers to not be
* shot.
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static String STORM_SUPERVISOR_HARD_LIMIT_MEMORY_OVERAGE_MB = "storm.supervisor.hard.memory.limit.overage.mb";
/**
* If the amount of memory that is free in the system (either on the box or in the supervisor's cgroup) is below this number (in MB)
* consider the system to be in low memory mode and start shooting workers if they are over their limit.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_SUPERVISOR_LOW_MEMORY_THRESHOLD_MB = "storm.supervisor.low.memory.threshold.mb";
/**
* If the amount of memory that is free in the system (either on the box or in the supervisor's cgroup) is below this number (in MB)
* consider the system to be a little low on memory and start shooting workers if they are over their limit for a given grace period
* STORM_SUPERVISOR_MEDIUM_MEMORY_GRACE_PERIOD_MS.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_SUPERVISOR_MEDIUM_MEMORY_THRESHOLD_MB = "storm.supervisor.medium.memory.threshold.mb";
/**
* The number of milliseconds that a worker is allowed to be over their limit when there is a medium amount of memory free in the
* system.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_SUPERVISOR_MEDIUM_MEMORY_GRACE_PERIOD_MS =
"storm.supervisor.medium.memory.grace.period.ms";
@@ -1147,7 +1147,7 @@
* Workers in containers or cgroups may require a minimum amount of CPU in order to launch within the supervisor timeout.
* This setting allows configuring this to occur.
*/
- @isPositiveNumber(includeZero = true)
+ @IsPositiveNumber(includeZero = true)
public static String STORM_WORKER_MIN_CPU_PCORE_PERCENT = "storm.worker.min.cpu.pcore.percent";
// VALIDATION ONLY CONFIGS
@@ -1158,7 +1158,7 @@
/**
* The number of hours a worker token is valid for. This also sets how frequently worker tokens will be renewed.
*/
- @isPositiveNumber
+ @IsPositiveNumber
public static String STORM_WORKER_TOKEN_LIFE_TIME_HOURS = "storm.worker.token.life.time.hours";
public static String getCgroupRootDir(Map<String, Object> conf) {
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
index 8b20008..96d6c21 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
@@ -143,9 +143,6 @@
import org.apache.storm.generated.WorkerMetrics;
import org.apache.storm.generated.WorkerResources;
import org.apache.storm.generated.WorkerSummary;
-import org.apache.storm.generated.WorkerToken;
-import org.apache.storm.generated.WorkerTokenInfo;
-import org.apache.storm.generated.WorkerTokenServiceType;
import org.apache.storm.logging.ThriftAccessLogger;
import org.apache.storm.metric.ClusterMetricsConsumerExecutor;
import org.apache.storm.metric.StormMetricsRegistry;