[storm] branch master updated: STORM-3512 Empty implementation for the MetricStore. (#3152)

2020-01-24 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 86e5b2c  STORM-3512 Empty implementation for the MetricStore.  (#3152)
86e5b2c is described below

commit 86e5b2c514b0f03fe566d95c1c0f695242d1027f
Author: tit4 
AuthorDate: Fri Jan 24 14:38:13 2020 +0300

STORM-3512 Empty implementation for the MetricStore.  (#3152)

* STORM-3512 Empty implementation for the MetricStore. Intended to be use 
in the config to disable monitoring.
---
 .../apache/storm/metricstore/NoOpMetricStore.java  | 40 ++
 1 file changed, 40 insertions(+)

diff --git 
a/storm-server/src/main/java/org/apache/storm/metricstore/NoOpMetricStore.java 
b/storm-server/src/main/java/org/apache/storm/metricstore/NoOpMetricStore.java
new file mode 100644
index 000..c9f67c1
--- /dev/null
+++ 
b/storm-server/src/main/java/org/apache/storm/metricstore/NoOpMetricStore.java
@@ -0,0 +1,40 @@
+/**
+ * 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.metricstore;
+
+import java.util.Map;
+import org.apache.storm.metric.StormMetricsRegistry;
+
+@SuppressWarnings("unused")
+public class NoOpMetricStore implements MetricStore {
+
+@Override
+public void prepare(Map config, StormMetricsRegistry 
metricsRegistry) {}
+
+@Override
+public void insert(Metric metric) { }
+
+@Override
+public boolean populateValue(Metric metric) {
+return true;
+}
+
+@Override
+public void close() { }
+
+@Override
+public void scan(FilterOptions filter, ScanCallback scanCallback) { }
+}
+
+
+
+



[storm] branch master updated (406f222 -> 1d946d3)

2019-11-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git.


from 406f222  STORM-3536 Add Generic-resources.md (#3166)
 add 1d946d3  MINOR: Add thrift.supervisor.port to SECURITY.md 
documentation (#3162)

No new revisions were added by this update.

Summary of changes:
 SECURITY.md  | 1 +
 docs/SECURITY.md | 1 +
 2 files changed, 2 insertions(+)



[storm] branch master updated (0c6648a -> 406f222)

2019-11-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git.


from 0c6648a  Merge pull request #3168 from dandsager1/STORM-3539
 add 406f222  STORM-3536 Add Generic-resources.md (#3166)

No new revisions were added by this update.

Summary of changes:
 docs/Generic-resources.md |  39 +++
 docs/images/storm_ui.png  | Bin 0 -> 1054600 bytes
 docs/index.md |   1 +
 3 files changed, 40 insertions(+)
 create mode 100644 docs/Generic-resources.md
 create mode 100644 docs/images/storm_ui.png



[storm] branch master updated: STORM-3529: Catch and Log Kafka RetriableException

2019-11-13 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b17e548  STORM-3529: Catch and Log Kafka RetriableException
 new dd8ce7b  Merge pull request #3164 from OliverMD/STORM-3529
b17e548 is described below

commit b17e5483cdecf19b3c6b6f5aa8bc7bedb1a23340
Author: Oliver Downard 
AuthorDate: Sun Nov 10 09:43:22 2019 +

STORM-3529: Catch and Log Kafka RetriableException
---
 .../kafka/spout/metrics/KafkaOffsetMetric.java  | 14 --
 .../kafka/spout/KafkaSpoutSingleTopicTest.java  | 21 -
 2 files changed, 28 insertions(+), 7 deletions(-)

diff --git 
a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
 
b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
index da84979..496e1d8 100644
--- 
a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
+++ 
b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/metrics/KafkaOffsetMetric.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import java.util.function.Supplier;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.RetriableException;
 import org.apache.storm.kafka.spout.internal.OffsetManager;
 import org.apache.storm.metric.api.IMetric;
 import org.slf4j.Logger;
@@ -76,8 +77,17 @@ public class KafkaOffsetMetric implements IMetric {
 Map topicMetricsMap = new HashMap<>();
 Set topicPartitions = offsetManagers.keySet();
 
-Map beginningOffsets = 
consumer.beginningOffsets(topicPartitions);
-Map endOffsets = 
consumer.endOffsets(topicPartitions);
+Map beginningOffsets;
+Map endOffsets;
+
+try {
+beginningOffsets = consumer.beginningOffsets(topicPartitions);
+endOffsets = consumer.endOffsets(topicPartitions);
+} catch (RetriableException e) {
+LOG.warn("Failed to get offsets from Kafka! Will retry on next 
metrics tick.", e);
+return null;
+}
+
 //map to hold partition level and topic level metrics
 Map result = new HashMap<>();
 
diff --git 
a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
 
b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
index 512d274..d7f563f 100644
--- 
a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
+++ 
b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutSingleTopicTest.java
@@ -21,17 +21,14 @@ package org.apache.storm.kafka.spout;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.anyListOf;
 import static org.mockito.ArgumentMatchers.anyObject;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.clearInvocations;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.*;
 
 import java.util.HashSet;
 import java.util.List;
@@ -39,8 +36,10 @@ import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.TimeoutException;
 import 
org.apache.storm.kafka.spout.config.builder.SingleTopicKafkaSpoutConfiguration;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Time;
@@ -428,4 +427,16 @@ public class KafkaSpoutSingleTopicTest extends 
KafkaSpoutAbstractTest {
 
assertEquals(offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalLatestCompletedOffset").longValue(),
 10);
 
assertEquals(offsetMetric.get(SingleTopicKafkaSpoutConfiguration.TOPIC+"/totalSpoutLag").longValue(),
 0);
 }
+
+@Test
+public void testOffsetMetricsReturnsNullWhenRetriableExceptionThrown() 
throws Exception {
+final int messageCount = 10;
+prepareSpout(messageCount);
+
+// Ensure a timeout exception results in 

[storm] branch master updated: STORM-3211: Fix NPE in WindowedBoltExecutor on getComponentConfiguration

2019-11-02 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 55f346c  STORM-3211: Fix NPE in WindowedBoltExecutor on 
getComponentConfiguration
 new 9165a78  Merge pull request #3156 from efgpinto/STORM-3211
55f346c is described below

commit 55f346cb093d4f219ecb43b1e6b1a7c2ea5d9a5f
Author: Eduardo Pinto 
AuthorDate: Sun Oct 27 22:25:02 2019 +

STORM-3211: Fix NPE in WindowedBoltExecutor on getComponentConfiguration
---
 .../src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java  | 3 ++-
 .../jvm/org/apache/storm/topology/WindowedBoltExecutorTest.java  | 9 +
 2 files changed, 11 insertions(+), 1 deletion(-)

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 e6a12d9..4713e95 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java
@@ -16,6 +16,7 @@ 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.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -345,7 +346,7 @@ public class WindowedBoltExecutor implements IRichBolt {
 
 @Override
 public Map getComponentConfiguration() {
-return bolt.getComponentConfiguration();
+return bolt.getComponentConfiguration() != null ? 
bolt.getComponentConfiguration() : Collections.emptyMap();
 }
 
 protected WindowLifecycleListener newWindowLifecycleListener() {
diff --git 
a/storm-client/test/jvm/org/apache/storm/topology/WindowedBoltExecutorTest.java 
b/storm-client/test/jvm/org/apache/storm/topology/WindowedBoltExecutorTest.java
index 5162cc6..0e38838 100644
--- 
a/storm-client/test/jvm/org/apache/storm/topology/WindowedBoltExecutorTest.java
+++ 
b/storm-client/test/jvm/org/apache/storm/topology/WindowedBoltExecutorTest.java
@@ -39,6 +39,7 @@ import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -214,6 +215,14 @@ public class WindowedBoltExecutorTest {
 Mockito.verify(outputCollector).emit("$late", Arrays.asList(tuple), 
new Values(tuple));
 }
 
+@Test
+public void testEmptyConfigOnWrappedBolt() {
+IWindowedBolt wrappedBolt = Mockito.mock(IWindowedBolt.class);
+Mockito.when(wrappedBolt.getComponentConfiguration()).thenReturn(null);
+executor = new WindowedBoltExecutor(wrappedBolt);
+assertTrue("Configuration is not empty", 
executor.getComponentConfiguration().isEmpty());
+}
+
 private static class TestWindowedBolt extends BaseWindowedBolt {
 List tupleWindows = new ArrayList<>();
 



[storm] branch master updated: STORM-3066: Implement support for using list elements in properties

2019-11-02 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b6ed6ad  STORM-3066: Implement support for using list elements in 
properties
 new dfb9b55  Merge pull request #3155 from efgpinto/STORM-3066
b6ed6ad is described below

commit b6ed6ade4f75863b2607d4478898449c469bb662
Author: Eduardo Pinto 
AuthorDate: Sun Oct 27 21:16:55 2019 +

STORM-3066: Implement support for using list elements in properties
---
 .../org/apache/storm/flux/parser/FluxParser.java   | 103 -
 .../test/java/org/apache/storm/flux/TCKTest.java   |   5 +
 .../test/resources/configs/substitution-test.yaml  |   2 +
 3 files changed, 68 insertions(+), 42 deletions(-)

diff --git 
a/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java 
b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
index 8299c14..50570e1 100644
--- a/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
@@ -18,12 +18,17 @@
 
 package org.apache.storm.flux.parser;
 
-import java.io.ByteArrayOutputStream;
+import java.io.BufferedReader;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import org.apache.storm.flux.model.BoltDef;
 import org.apache.storm.flux.model.IncludeDef;
@@ -40,17 +45,20 @@ import org.yaml.snakeyaml.constructor.Constructor;
  */
 public class FluxParser {
 private static final Logger LOG = 
LoggerFactory.getLogger(FluxParser.class);
+private static final Pattern propertyPattern =
+
Pattern.compile(".*\\$\\{(?ENV-(?.+)|(?.+)\\[(?\\d+)]|.+)}.*");
 
 private FluxParser() {
 }
 
 /**
  * Parse a flux topology definition.
- * @param inputFile source YAML file
- * @param dumpYaml if true, dump the parsed YAML to stdout
+ *
+ * @param inputFile   source YAML file
+ * @param dumpYamlif true, dump the parsed YAML to stdout
  * @param processIncludes whether or not to process includes
- * @param properties properties file for variable substitution
- * @param envSub whether or not to perform environment variable 
substitution
+ * @param properties  properties file for variable substitution
+ * @param envSub  whether or not to perform environment variable 
substitution
  * @return resulting topologuy definition
  * @throws IOException if there is a problem reading file(s)
  */
@@ -65,11 +73,12 @@ public class FluxParser {
 
 /**
  * Parse a flux topology definition from a classpath resource..
- * @param resource YAML resource
- * @param dumpYaml if true, dump the parsed YAML to stdout
+ *
+ * @param resourceYAML resource
+ * @param dumpYamlif true, dump the parsed YAML to stdout
  * @param processIncludes whether or not to process includes
- * @param properties properties file for variable substitution
- * @param envSub whether or not to perform environment variable 
substitution
+ * @param properties  properties file for variable substitution
+ * @param envSub  whether or not to perform environment variable 
substitution
  * @return resulting topologuy definition
  * @throws IOException if there is a problem reading file(s)
  */
@@ -84,11 +93,12 @@ public class FluxParser {
 
 /**
  * Parse a flux topology definition.
- * @param inputStream InputStream representation of YAML file
- * @param dumpYaml if true, dump the parsed YAML to stdout
+ *
+ * @param inputStream InputStream representation of YAML file
+ * @param dumpYamlif true, dump the parsed YAML to stdout
  * @param processIncludes whether or not to process includes
- * @param properties properties file for variable substitution
- * @param envSub whether or not to perform environment variable 
substitution
+ * @param properties  properties file for variable substitution
+ * @param envSub  whether or not to perform environment variable 
substitution
  * @return resulting topology definition
  * @throws IOException if there is a problem reading file(s)
  */
@@ -116,10 +126,11 @@ public class FluxParser {
 
 /**
  * Parse filter properties file.
+ *
  * @param propertiesFile properties file for variable substitution
- * @param resource whether or not to load properties file from classpath
+ * @param resource   whether or not to load properties file from 
classpath
 

[storm] branch master updated: STORM-2749 Remove unimplemented setStateSpout function

2019-10-24 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 7d45bc3  STORM-2749 Remove unimplemented setStateSpout function
 new 4fffbd9  Merge pull request #3143 from daisyxchen/STORM-2749
7d45bc3 is described below

commit 7d45bc381c9e2b31cedd7aa113aa2829b88e8a97
Author: daisyxchen 
AuthorDate: Mon Oct 14 14:10:12 2019 -0500

STORM-2749 Remove unimplemented setStateSpout function
---
 .../src/jvm/org/apache/storm/topology/TopologyBuilder.java   | 9 -
 .../test/jvm/org/apache/storm/topology/TopologyBuilderTest.java  | 6 --
 2 files changed, 15 deletions(-)

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 63384df..c14d6fb 100644
--- a/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java
+++ b/storm-client/src/jvm/org/apache/storm/topology/TopologyBuilder.java
@@ -473,15 +473,6 @@ public class TopologyBuilder {
 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.
  *
diff --git 
a/storm-client/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java 
b/storm-client/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java
index 3e8a86d..47fe12f 100644
--- a/storm-client/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java
+++ b/storm-client/test/jvm/org/apache/storm/topology/TopologyBuilderTest.java
@@ -51,12 +51,6 @@ public class TopologyBuilderTest {
 builder.addWorkerHook(null);
 }
 
-// TODO enable if setStateSpout gets implemented
-//@Test(expected = IllegalArgumentException.class)
-//public void testSetStateSpout() {
-//builder.setStateSpout("stateSpout", mock(IRichStateSpout.class), 
0);
-//}
-
 @Test
 public void testStatefulTopology() {
 builder.setSpout("spout1", makeDummySpout());



[storm] branch master updated: STORM-3510: Track overflow count per taskId for resending backpressur… (#3131)

2019-10-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new c78f1aa  STORM-3510: Track overflow count per taskId for resending 
backpressur… (#3131)
c78f1aa is described below

commit c78f1aaa8fcd17eb5c8f45f5dbb90e30f99e19bb
Author: cjljohnson 
AuthorDate: Tue Oct 1 15:38:28 2019 +0100

STORM-3510: Track overflow count per taskId for resending backpressur… 
(#3131)

* STORM-3510: Track overflow count per taskId for resending backpressure 
status
---
 .../storm/daemon/worker/BackPressureTracker.java   | 29 ++
 .../apache/storm/daemon/worker/WorkerState.java| 14 +
 .../daemon/worker/BackPressureTrackerTest.java | 35 ++
 3 files changed, 60 insertions(+), 18 deletions(-)

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 dae5cca..3c590e5 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
@@ -49,8 +49,12 @@ public class BackPressureTracker {
 entry -> new BackpressureState(entry.getValue(;
 }
 
-private void recordNoBackPressure(Integer taskId) {
-tasks.get(taskId).backpressure.set(false);
+public BackpressureState getBackpressureState(Integer taskId) {
+return tasks.get(taskId);
+}
+
+private void recordNoBackPressure(BackpressureState state) {
+state.backpressure.set(false);
 }
 
 /**
@@ -60,8 +64,8 @@ public class BackPressureTracker {
  *
  * @return true if an update was recorded, false if taskId is already 
under BP
  */
-public boolean recordBackPressure(Integer taskId) {
-return tasks.get(taskId).backpressure.getAndSet(true) == false;
+public boolean recordBackPressure(BackpressureState state) {
+return state.backpressure.getAndSet(true) == false;
 }
 
 // returns true if there was a change in the BP situation
@@ -71,7 +75,7 @@ public class BackPressureTracker {
 for (Entry entry : tasks.entrySet()) {
 BackpressureState state = entry.getValue();
 if (state.backpressure.get() && state.queue.isEmptyOverflow()) {
-recordNoBackPressure(entry.getKey());
+recordNoBackPressure(state);
 changed = true;
 }
 }
@@ -95,11 +99,24 @@ public class BackPressureTracker {
 }
 return new BackPressureStatus(workerId, bpTasks, nonBpTasks);
 }
+
+public int getLastOverflowCount(BackpressureState state) {
+return state.lastOverflowCount;
+}
+
+public void setLastOverflowCount(BackpressureState state, int value) {
+state.lastOverflowCount = value;
+}
+
+
 
-private static class BackpressureState {
+public static class BackpressureState {
 private final JCQueue queue;
 //No task is under backpressure initially
 private final AtomicBoolean backpressure = new AtomicBoolean(false);
+//The overflow count last time BP status was sent
+private int lastOverflowCount = 0;
+
 
 BackpressureState(JCQueue queue) {
 this.queue = queue;
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 f380769..eaab4e9 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
@@ -42,6 +42,7 @@ import org.apache.storm.cluster.IStormClusterState;
 import org.apache.storm.cluster.VersionedData;
 import org.apache.storm.daemon.StormCommon;
 import org.apache.storm.daemon.supervisor.AdvancedFSOps;
+import org.apache.storm.daemon.worker.BackPressureTracker.BackpressureState;
 import org.apache.storm.executor.IRunningExecutor;
 import org.apache.storm.generated.Assignment;
 import org.apache.storm.generated.DebugOptions;
@@ -88,6 +89,7 @@ public class WorkerState {
 
 private static final Logger LOG = 
LoggerFactory.getLogger(WorkerState.class);
 private static final long LOAD_REFRESH_INTERVAL_MS = 5000L;
+private static final int RESEND_BACKPRESSURE_SIZE = 1;
 private static long dropCount = 0;
 final Map conf;
 final IContext mqContext;
@@ -533,8 +535,6 @@ public class WorkerState {
 // Receives msgs from remote workers and feeds them to local executors. If 
any receiving local executor is under Back Pressure,
 // informs other workers about back pressure situation. Runs in the 
NettyWorker thread.
 private void transferLocalBatch(ArrayList tupleBatch) {
-int lastOverfl

[storm-site] branch asf-site updated (6a0544c -> 8b325f0)

2019-09-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git.


from 6a0544c  Merge pull request #11 from Ethanlm/update-storm-reference
 new 0ad130c  Update README so command to build Storm javadoc actually 
works, bump ruby gems
 new 9ab5d2e  STORM-3508: Fix broken link on site
 new 90dd905  Merge branch 'asf-site' of 
https://git-wip-us.apache.org/repos/asf/storm-site into asfgit-site
 new 8b325f0  Rebuild site

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 Gemfile.lock   | 2 +-
 README.md  | 4 +-
 content/2012/08/02/storm080-released.html  |   488 +-
 content/2012/09/06/storm081-released.html  |   488 +-
 content/2013/01/11/storm082-released.html  |   488 +-
 content/2013/12/08/storm090-released.html  |   488 +-
 content/2014/04/10/storm-logo-contest.html |   488 +-
 content/2014/04/17/logo-pforrest.html  |   488 +-
 content/2014/04/17/logo-squinones.html |   488 +-
 content/2014/04/19/logo-ssuleman.html  |   488 +-
 content/2014/04/21/logo-rmarshall.html |   488 +-
 content/2014/04/22/logo-zsayari.html   |   488 +-
 content/2014/04/23/logo-abartos.html   |   488 +-
 content/2014/04/27/logo-cboustead.html |   488 +-
 content/2014/04/27/logo-sasili.html|   488 +-
 content/2014/04/29/logo-jlee1.html |   488 +-
 content/2014/04/29/logo-jlee2.html |   488 +-
 content/2014/04/29/logo-jlee3.html |   488 +-
 content/2014/05/27/round1-results.html |   488 +-
 content/2014/06/17/contest-results.html|   488 +-
 content/2014/06/25/storm092-released.html  |   488 +-
 content/2014/10/20/storm093-release-candidate.html |   488 +-
 content/2014/11/25/storm093-released.html  |   488 +-
 content/2015/03/25/storm094-released.html  |   488 +-
 content/2015/06/04/storm095-released.html  |   488 +-
 content/2015/06/15/storm0100-beta-released.html|   488 +-
 content/2015/11/05/storm0100-released.html |   488 +-
 content/2015/11/05/storm096-released.html  |   488 +-
 content/2016/04/12/storm100-released.html  |   488 +-
 content/2016/05/05/storm0101-released.html |   488 +-
 content/2016/05/06/storm101-released.html  |   488 +-
 content/2016/08/10/storm102-released.html  |   488 +-
 content/2016/09/07/storm097-released.html  |   488 +-
 content/2016/09/14/storm0102-released.html |   488 +-
 content/2017/02/14/storm103-released.html  |   488 +-
 content/2017/03/29/storm110-released.html  |   488 +-
 content/2017/07/28/storm104-released.html  |   488 +-
 content/2017/08/01/storm111-released.html  |   488 +-
 content/2017/09/15/storm105-released.html  |   488 +-
 content/2018/02/14/storm106-released.html  |   488 +-
 content/2018/02/15/storm112-released.html  |   488 +-
 content/2018/02/15/storm120-released.html  |   488 +-
 content/2018/02/19/storm121-released.html  |   488 +-
 content/2018/06/04/storm113-released.html  |   488 +-
 content/2018/06/04/storm122-released.html  |   488 +-
 content/2019/05/30/storm200-released.html  |   488 +-
 content/2019/07/18/storm123-released.html  |   488 +-
 content/Gemfile| 8 +-
 content/Gemfile.lock   | 2 +-
 content/Powered-By.html|   238 +-
 content/about/deployment.html  |   314 +-
 content/about/fault-tolerant.html  |   314 +-
 content/about/free-and-open-source.html|   314 +-
 content/about/guarantees-data-processing.html  |   314 +-
 content/about/integrates.html  |   314 +-
 content/about/multi-language.html  |   314 +-
 content/about/scalable.html|   314 +-
 content/about/simple-api.html  |   314 +-
 content/assets/css/bootstrap-theme.css |   940 +-
 content/assets/css/bootstrap-theme.min.css | 8 +-
 content/assets/css/bootstrap.css   | 13600 +--
 content/assets/css/bootstrap.min.css   | 8 +-
 content/assets/css/font-awesome.min.css| 6 +-
 content/assets/css/owl.carousel.css|   142 +-
 content/assets/css/owl.theme.css   |   156 +-
 content/assets/css/style.css   |  1006 +-

[storm-site] 03/04: Merge branch 'asf-site' of https://git-wip-us.apache.org/repos/asf/storm-site into asfgit-site

2019-09-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit 90dd9056eef08ddf4642f90863d54d45324218db
Merge: 9ab5d2e 6a0544c
Author: Stig Rohde Døssing 
AuthorDate: Sat Sep 21 09:36:38 2019 +0200

Merge branch 'asf-site' of 
https://git-wip-us.apache.org/repos/asf/storm-site into asfgit-site

 _includes/footer.html  |   6 +-
 _posts/2012-08-02-storm080-released.md |  22 ++--
 _posts/2012-09-06-storm081-released.md |   8 +-
 _posts/2013-01-11-storm082-released.md |  12 +-
 _posts/2013-12-08-storm090-released.md |  34 +++---
 _posts/2014-04-10-storm-logo-contest.md|   8 +-
 _posts/2014-05-27-round1-results.md|   4 +-
 _posts/2014-06-17-contest-results.md   |   8 +-
 _posts/2014-06-25-storm092-released.md |  32 +++---
 _posts/2014-10-20-storm093-release-candidate.md|   6 +-
 _posts/2014-11-25-storm093-released.md |  18 +--
 _posts/2015-03-25-storm094-released.md |   4 +-
 _posts/2015-06-04-storm095-released.md |   4 +-
 _posts/2015-06-15-storm0100-beta-released.md   |  52 -
 _posts/2015-11-05-storm0100-released.md|   2 +-
 _posts/2015-11-05-storm096-released.md |   4 +-
 _posts/2016-04-12-storm100-released.md |  42 +++
 _posts/2016-05-05-storm0101-released.md|   4 +-
 _posts/2016-05-06-storm101-released.md |   6 +-
 _posts/2016-08-10-storm102-released.md |   4 +-
 _posts/2016-09-07-storm097-released.md |   6 +-
 _posts/2016-09-14-storm0102-released.md|   4 +-
 _posts/2017-02-14-storm103-released.md |   4 +-
 _posts/2017-03-29-storm110-released.md |  36 +++---
 _posts/2017-07-28-storm104-released.md |   4 +-
 _posts/2017-08-01-storm111-released.md |   4 +-
 _posts/2017-09-15-storm105-released.md |   4 +-
 _posts/2018-02-14-storm106-released.md |   6 +-
 _posts/2018-02-15-storm112-released.md |   6 +-
 _posts/2018-02-15-storm120-released.md |   6 +-
 _posts/2018-02-19-storm121-released.md |   6 +-
 _posts/2018-06-04-storm113-released.md |   4 +-
 _posts/2018-06-04-storm122-released.md |   4 +-
 _posts/2019-05-30-storm200-released.md |  26 ++---
 _posts/2019-07-18-storm123-released.md |   4 +-
 about/deployment.md|   4 +-
 about/fault-tolerant.md|   6 +-
 about/free-and-open-source.md  |   4 +-
 about/guarantees-data-processing.md|   6 +-
 about/integrates.md|   4 +-
 about/multi-language.md|   4 +-
 about/scalable.md  |   4 +-
 about/simple-api.md|   8 +-
 content/2012/08/02/storm080-released.html  |  94 
 content/2012/09/06/storm081-released.html  |  80 ++---
 content/2013/01/11/storm082-released.html  |  84 +++---
 content/2013/12/08/storm090-released.html  | 106 +-
 content/2014/04/10/storm-logo-contest.html |  78 ++---
 content/2014/04/17/logo-pforrest.html  |  70 ++--
 content/2014/04/17/logo-squinones.html |  70 ++--
 content/2014/04/19/logo-ssuleman.html  |  70 ++--
 content/2014/04/21/logo-rmarshall.html |  70 ++--
 content/2014/04/22/logo-zsayari.html   |  70 ++--
 content/2014/04/23/logo-abartos.html   |  70 ++--
 content/2014/04/27/logo-cboustead.html |  70 ++--
 content/2014/04/27/logo-sasili.html|  70 ++--
 content/2014/04/29/logo-jlee1.html |  70 ++--
 content/2014/04/29/logo-jlee2.html |  70 ++--
 content/2014/04/29/logo-jlee3.html |  70 ++--
 content/2014/05/27/round1-results.html |  74 ++--
 content/2014/06/17/contest-results.html|  80 ++---
 content/2014/06/25/storm092-released.html  | 104 -
 content/2014/10/20/storm093-release-candidate.html |  78 ++---
 content/2014/11/25/storm093-released.html  |  90 +++
 content/2015/03/25/storm094-released.html  |  76 ++---
 content/2015/06/04/storm095-released.html  |  76 ++---
 content/2015/06/15/storm0100-beta-released.html| 124 ++---
 content/2015/11/05/storm0100-released.html |  74 ++--
 content/2015/11/05/storm096-released.html  |  76 ++---
 content/2016/04/12/storm100-released.html

[storm-site] 02/04: STORM-3508: Fix broken link on site

2019-09-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit 9ab5d2e26883565f2d7a67b41fd4debc0782e76d
Author: Stig Rohde Døssing 
AuthorDate: Sat Sep 21 09:21:55 2019 +0200

STORM-3508: Fix broken link on site
---
 releases/1.2.3/Setting-up-development-environment.md | 2 +-
 releases/2.0.0/Setting-up-development-environment.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/releases/1.2.3/Setting-up-development-environment.md 
b/releases/1.2.3/Setting-up-development-environment.md
index bfa98a2..72e3472 100644
--- a/releases/1.2.3/Setting-up-development-environment.md
+++ b/releases/1.2.3/Setting-up-development-environment.md
@@ -5,7 +5,7 @@ documentation: true
 ---
 This page outlines what you need to do to get a Storm development environment 
set up. In summary, the steps are:
 
-1. Download a [Storm release](..//downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
+1. Download a [Storm release](../../downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
 2. To be able to start and stop topologies on a remote cluster, put the 
cluster information in `~/.storm/storm.yaml`
 
 More detail on each of these steps is below.
diff --git a/releases/2.0.0/Setting-up-development-environment.md 
b/releases/2.0.0/Setting-up-development-environment.md
index bfa98a2..72e3472 100644
--- a/releases/2.0.0/Setting-up-development-environment.md
+++ b/releases/2.0.0/Setting-up-development-environment.md
@@ -5,7 +5,7 @@ documentation: true
 ---
 This page outlines what you need to do to get a Storm development environment 
set up. In summary, the steps are:
 
-1. Download a [Storm release](..//downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
+1. Download a [Storm release](../../downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
 2. To be able to start and stop topologies on a remote cluster, put the 
cluster information in `~/.storm/storm.yaml`
 
 More detail on each of these steps is below.



[storm-site] 01/04: Update README so command to build Storm javadoc actually works, bump ruby gems

2019-09-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit 0ad130cd84fc618d7f88635191f2a539eaa90492
Author: Stig Rohde Døssing 
AuthorDate: Sat Sep 21 09:05:19 2019 +0200

Update README so command to build Storm javadoc actually works, bump ruby 
gems
---
 Gemfile.lock | 2 +-
 README.md| 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/Gemfile.lock b/Gemfile.lock
index 2250684..b949c38 100644
--- a/Gemfile.lock
+++ b/Gemfile.lock
@@ -237,4 +237,4 @@ DEPENDENCIES
   redcarpet (~> 3.2, >= 3.2.3)
 
 BUNDLED WITH
-   1.16.1
+   2.1.0.pre.1
diff --git a/README.md b/README.md
index eaf17dc..839e6fc 100644
--- a/README.md
+++ b/README.md
@@ -29,9 +29,11 @@ In order to add a new release, you must have committer 
access to the storm-site
 
 You must first generate Javadoc for the new release. Check out the Storm 
repository from https://github.com/apache/storm, and check out the version of 
the code you are releasing.
 
+You must have already installed the storm-shaded-deps module, so please run 
`mvn clean install -pl storm-shaded-deps -am` if you haven't built Storm 
already.
+
 In the Storm project root run
 ```
-mvn javadoc:aggregate -DreportOutputDirectory=./docs/ -DdestDir=javadocs 
-Dnotimestamp=true -P '!include-shaded-deps'
+mvn javadoc:aggregate -DreportOutputDirectory=./docs/ -DdestDir=javadocs 
-Dnotimestamp=true -pl '!storm-shaded-deps'
 ```
 
 In the storm-site project, release documentation is placed under the releases 
directory named after the release version. See 
[below](#how-release-specific-docs-work) for details about release specific 
documentation.



[storm] branch master updated: STORM-3508: Fix broken link on site (#3130)

2019-09-21 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 38b09fd  STORM-3508: Fix broken link on site (#3130)
38b09fd is described below

commit 38b09fd229d2034a0fb6d5359eaa0b8ecdc0b61a
Author: Stig Døssing 
AuthorDate: Sat Sep 21 09:01:48 2019 +0200

STORM-3508: Fix broken link on site (#3130)
---
 docs/Setting-up-development-environment.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/Setting-up-development-environment.md 
b/docs/Setting-up-development-environment.md
index bfa98a2..72e3472 100644
--- a/docs/Setting-up-development-environment.md
+++ b/docs/Setting-up-development-environment.md
@@ -5,7 +5,7 @@ documentation: true
 ---
 This page outlines what you need to do to get a Storm development environment 
set up. In summary, the steps are:
 
-1. Download a [Storm release](..//downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
+1. Download a [Storm release](../../downloads.html) , unpack it, and put the 
unpacked `bin/` directory on your PATH
 2. To be able to start and stop topologies on a remote cluster, put the 
cluster information in `~/.storm/storm.yaml`
 
 More detail on each of these steps is below.



[storm] branch master updated (93b7d7c -> 74958a6)

2019-09-12 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git.


from 93b7d7c  [STORM-3503] add unit test for 
BLACKLIST_SCHEDULER_ASSUME_SUPERVISOR_… (#3122)
 add 74958a6  STORM-3504: changed AsyncLocalizerTest to not mock 
AdvancedFSOps (#3126)

No new revisions were added by this update.

Summary of changes:
 .../apache/storm/localizer/AsyncLocalizerTest.java | 1010 ++--
 1 file changed, 496 insertions(+), 514 deletions(-)



[storm] branch 2.1.x-branch updated: STORM-1515: Fix LocalState Corruption

2019-09-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 2.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/2.1.x-branch by this push:
 new 0e5a68e  STORM-1515: Fix LocalState Corruption
0e5a68e is described below

commit 0e5a68e9d2e8d5e79a1af36203f52801ba6d4864
Author: Tim Frison 
AuthorDate: Tue Aug 13 11:28:31 2019 -0600

STORM-1515: Fix LocalState Corruption

When a windows machine has a power failure, the local state file can
become corrupted with repeated NUL characters. On restart, when the
supervisor attempts to get the worker's heartbeat it will fail to
deserialized the LocalStateData (because it is all NUL characters) and
it will fail to start the workers.
---
 .../src/jvm/org/apache/storm/utils/LocalState.java |  5 +++
 .../test/java/org/apache/storm/LocalStateTest.java | 40 +-
 2 files changed, 36 insertions(+), 9 deletions(-)

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 6e1b44f..d5996d2 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
@@ -32,6 +32,7 @@ import org.apache.storm.shade.org.apache.commons.io.FileUtils;
 import org.apache.storm.thrift.TBase;
 import org.apache.storm.thrift.TDeserializer;
 import org.apache.storm.thrift.TSerializer;
+import org.apache.storm.thrift.protocol.TProtocolException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +124,10 @@ public class LocalState {
 } catch (Exception e) {
 attempts++;
 if (attempts >= 10) {
+if (e.getCause() instanceof TProtocolException) {
+LOG.warn("LocalState file is corrupted, resetting 
state.", e);
+return new HashMap<>();
+}
 throw new RuntimeException(e);
 }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/LocalStateTest.java 
b/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
index 97765b2..2a14857 100644
--- a/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
+++ b/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
@@ -57,15 +57,37 @@ public class LocalStateTest {
 
 @Test
 public void testEmptyState() throws IOException {
-TmpPath tmp_dir = new TmpPath();
-String dir = tmp_dir.getPath();
-LocalState ls = new LocalState(dir, true);
-GlobalStreamId gs_a = new GlobalStreamId("a", "a");
-FileOutputStream data = FileUtils.openOutputStream(new File(dir, 
"12345"));
-FileOutputStream version = FileUtils.openOutputStream(new File(dir, 
"12345.version"));
-Assert.assertNull(ls.get("c"));
-ls.put("a", gs_a);
-Assert.assertEquals(gs_a, ls.get("a"));
+try (TmpPath tmp_dir = new TmpPath()) {
+GlobalStreamId globalStreamId_a = new GlobalStreamId("a", "a");
+
+String dir = tmp_dir.getPath();
+LocalState ls = new LocalState(dir, true);
+
+FileUtils.touch(new File(dir, "12345"));
+FileUtils.touch(new File(dir, "12345.version"));
+
+Assert.assertNull(ls.get("c"));
+ls.put("a", globalStreamId_a);
+Assert.assertEquals(globalStreamId_a, ls.get("a"));
+}
+}
 
+@Test
+public void testAllNulState() throws IOException {
+try (TmpPath tmp_dir = new TmpPath()) {
+GlobalStreamId globalStreamId_a = new GlobalStreamId("a", "a");
+
+String dir = tmp_dir.getPath();
+LocalState ls = new LocalState(dir, true);
+
+FileUtils.touch(new File(dir, "12345.version"));
+
+try (FileOutputStream data = FileUtils.openOutputStream(new 
File(dir, "12345"))) {
+Assert.assertNull(ls.get("c"));
+data.write(new byte[100]);
+ls.put("a", globalStreamId_a);
+Assert.assertEquals(globalStreamId_a, ls.get("a"));
+}
+}
 }
 }



[storm] branch master updated: STORM-1515: Fix LocalState Corruption

2019-09-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new d212b59  STORM-1515: Fix LocalState Corruption
 new 23cd491  Merge pull request #3101 from frison/STORM-1515
d212b59 is described below

commit d212b59cd05d2c67a9b09d0c6a98d241f77824fb
Author: Tim Frison 
AuthorDate: Tue Aug 13 11:28:31 2019 -0600

STORM-1515: Fix LocalState Corruption

When a windows machine has a power failure, the local state file can
become corrupted with repeated NUL characters. On restart, when the
supervisor attempts to get the worker's heartbeat it will fail to
deserialized the LocalStateData (because it is all NUL characters) and
it will fail to start the workers.
---
 .../src/jvm/org/apache/storm/utils/LocalState.java |  5 +++
 .../test/java/org/apache/storm/LocalStateTest.java | 40 +-
 2 files changed, 36 insertions(+), 9 deletions(-)

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 6e1b44f..d5996d2 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/LocalState.java
@@ -32,6 +32,7 @@ import org.apache.storm.shade.org.apache.commons.io.FileUtils;
 import org.apache.storm.thrift.TBase;
 import org.apache.storm.thrift.TDeserializer;
 import org.apache.storm.thrift.TSerializer;
+import org.apache.storm.thrift.protocol.TProtocolException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +124,10 @@ public class LocalState {
 } catch (Exception e) {
 attempts++;
 if (attempts >= 10) {
+if (e.getCause() instanceof TProtocolException) {
+LOG.warn("LocalState file is corrupted, resetting 
state.", e);
+return new HashMap<>();
+}
 throw new RuntimeException(e);
 }
 }
diff --git a/storm-server/src/test/java/org/apache/storm/LocalStateTest.java 
b/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
index 97765b2..2a14857 100644
--- a/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
+++ b/storm-server/src/test/java/org/apache/storm/LocalStateTest.java
@@ -57,15 +57,37 @@ public class LocalStateTest {
 
 @Test
 public void testEmptyState() throws IOException {
-TmpPath tmp_dir = new TmpPath();
-String dir = tmp_dir.getPath();
-LocalState ls = new LocalState(dir, true);
-GlobalStreamId gs_a = new GlobalStreamId("a", "a");
-FileOutputStream data = FileUtils.openOutputStream(new File(dir, 
"12345"));
-FileOutputStream version = FileUtils.openOutputStream(new File(dir, 
"12345.version"));
-Assert.assertNull(ls.get("c"));
-ls.put("a", gs_a);
-Assert.assertEquals(gs_a, ls.get("a"));
+try (TmpPath tmp_dir = new TmpPath()) {
+GlobalStreamId globalStreamId_a = new GlobalStreamId("a", "a");
+
+String dir = tmp_dir.getPath();
+LocalState ls = new LocalState(dir, true);
+
+FileUtils.touch(new File(dir, "12345"));
+FileUtils.touch(new File(dir, "12345.version"));
+
+Assert.assertNull(ls.get("c"));
+ls.put("a", globalStreamId_a);
+Assert.assertEquals(globalStreamId_a, ls.get("a"));
+}
+}
 
+@Test
+public void testAllNulState() throws IOException {
+try (TmpPath tmp_dir = new TmpPath()) {
+GlobalStreamId globalStreamId_a = new GlobalStreamId("a", "a");
+
+String dir = tmp_dir.getPath();
+LocalState ls = new LocalState(dir, true);
+
+FileUtils.touch(new File(dir, "12345.version"));
+
+try (FileOutputStream data = FileUtils.openOutputStream(new 
File(dir, "12345"))) {
+Assert.assertNull(ls.get("c"));
+data.write(new byte[100]);
+ls.put("a", globalStreamId_a);
+Assert.assertEquals(globalStreamId_a, ls.get("a"));
+}
+}
 }
 }



[storm] branch master updated: STORM-3486: Bump Jersey to 2.29 for JDK 11 support (#3103)

2019-09-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new f33b7d8  STORM-3486: Bump Jersey to 2.29 for JDK 11 support (#3103)
f33b7d8 is described below

commit f33b7d80b3a30b5f08cbcd36aa92b5e86b90263f
Author: Stig Døssing 
AuthorDate: Sun Sep 1 13:04:24 2019 +0200

STORM-3486: Bump Jersey to 2.29 for JDK 11 support (#3103)

* STORM-3486: Bump Jersey to 2.29 for JDK 11 support
---
 DEPENDENCY-LICENSES |  77 +
 LICENSE-binary  |  62 ++---
 external/storm-opentsdb/pom.xml |  24 +++
 pom.xml |  65 +++---
 storm-client/pom.xml|  14 +++-
 storm-webapp/pom.xml| 147 +++-
 6 files changed, 158 insertions(+), 231 deletions(-)

diff --git a/DEPENDENCY-LICENSES b/DEPENDENCY-LICENSES
index 3761367..e0344fc 100644
--- a/DEPENDENCY-LICENSES
+++ b/DEPENDENCY-LICENSES
@@ -202,7 +202,7 @@ List of third-party dependencies grouped by their license 
type.
 * AWS Java SDK for the AWS Simple Systems Management (SSM) Service 
(com.amazonaws:aws-java-sdk-ssm:1.10.77 - https://aws.amazon.com/sdkforjava)
 * AWS SDK For Java (com.amazonaws:aws-java-sdk:1.10.77 - 
https://aws.amazon.com/sdkforjava)
 * AWS SDK for Java - Core (com.amazonaws:aws-java-sdk-core:1.10.77 - 
https://aws.amazon.com/sdkforjava)
-* Bean Validation API (javax.validation:validation-api:1.1.0.Final - 
http://beanvalidation.org)
+* Bean Validation API (javax.validation:validation-api:2.0.1.Final - 
http://beanvalidation.org)
 * BoneCP :: Core Library (com.jolbox:bonecp:0.8.0.RELEASE - 
http://jolbox.com/bonecp)
 * Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.5 - 
https://github.com/ben-manes/caffeine)
 * Calcite Core (org.apache.calcite:calcite-core:1.14.0 - 
https://calcite.apache.org/calcite-core)
@@ -285,6 +285,7 @@ List of third-party dependencies grouped by their license 
type.
 * hawtdispatch (org.fusesource.hawtdispatch:hawtdispatch:1.22 - 
http://hawtdispatch.fusesource.org/hawtdispatch/)
 * hawtdispatch-transport 
(org.fusesource.hawtdispatch:hawtdispatch-transport:1.22 - 
http://hawtdispatch.fusesource.org/hawtdispatch-transport/)
 * Hibernate Validator Engine 
(org.hibernate:hibernate-validator:5.4.2.Final - 
http://hibernate.org/validator/hibernate-validator)
+* Hibernate Validator Engine 
(org.hibernate.validator:hibernate-validator:6.0.17.Final - 
http://hibernate.org/validator/hibernate-validator)
 * HikariCP (com.zaxxer:HikariCP:2.4.7 - 
https://github.com/brettwooldridge/HikariCP)
 * HikariCP (com.zaxxer:HikariCP:2.5.1 - 
https://github.com/brettwooldridge/HikariCP)
 * Hive CLI (org.apache.hive:hive-cli:2.3.4 - 
http://hive.apache.org/hive-cli)
@@ -483,29 +484,31 @@ List of third-party dependencies grouped by their license 
type.
 * zookeeper (org.apache.zookeeper:zookeeper:3.4.14 - no url defined)
 * zookeeper (org.apache.zookeeper:zookeeper:3.4.6 - no url defined)
 
-Apache License, Version 2.0, Common Development and Distribution License 
(CDDL) v1.1, Modified BSD, Public Domain, The GNU General Public License (GPL), 
Version 2, With Classpath Exception
+Apache License, Version 2.0, BSD 2-Clause, Eclipse Distribution License, 
Version 1.0, Eclipse Public License, Version 2.0, jQuery license, MIT License, 
Modified BSD, Public Domain, The GNU General Public License (GPL), Version 2, 
With Classpath Exception, W3C license
 
-* jersey-connectors-apache 
(org.glassfish.jersey.connectors:jersey-apache-connector:2.27 - 
https://jersey.github.io/project/jersey-apache-connector/)
-* jersey-container-grizzly2-http 
(org.glassfish.jersey.containers:jersey-container-grizzly2-http:2.27 - 
https://jersey.github.io/project/jersey-container-grizzly2-http/)
-* jersey-container-grizzly2-servlet 
(org.glassfish.jersey.containers:jersey-container-grizzly2-servlet:2.27 - 
https://jersey.github.io/project/jersey-container-grizzly2-servlet/)
-* jersey-container-servlet 
(org.glassfish.jersey.containers:jersey-container-servlet:2.27 - 
https://jersey.github.io/project/jersey-container-servlet/)
-* jersey-container-servlet-core 
(org.glassfish.jersey.containers:jersey-container-servlet-core:2.27 - 
https://jersey.github.io/project/jersey-container-servlet-core/)
-* jersey-core-client (org.glassfish.jersey.core:jersey-client:2.27 - 
https://jersey.github.io/jersey-client/)
-* jersey-ext-entity-filtering 
(org.glassfish.jersey.ext:jersey-entity-filtering:2.27 - 
https://jersey.github.io/project/jersey-entity-filtering/)
-* jersey-inject-hk2 (org.glassfish.jersey.inject:jersey-hk2:2.27 - 
https

[storm] branch 2.1.x-branch updated: [STORM-3501] local cluster worker restarts because of missing resources folder (#3120)

2019-08-31 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 2.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/2.1.x-branch by this push:
 new 71492f5  [STORM-3501] local cluster worker restarts because of missing 
resources folder (#3120)
71492f5 is described below

commit 71492f51789e6b7b0c842ae6a6bac48dc03f645a
Author: Diogo Monteiro 
AuthorDate: Sat Aug 31 18:19:14 2019 +0100

[STORM-3501] local cluster worker restarts because of missing resources 
folder (#3120)

* [STORM-3501] local cluster worker restarts because of missing resources 
folder
---
 .../storm/daemon/supervisor/AdvancedFSOps.java | 12 +++
 .../storm/daemon/supervisor/IAdvancedFSOps.java| 10 +++
 .../storm/localizer/LocallyCachedTopologyBlob.java |  9 +-
 .../java/org/apache/storm/utils/ServerUtils.java   | 14 +++
 .../apache/storm/localizer/AsyncLocalizerTest.java | 99 ++
 5 files changed, 141 insertions(+), 3 deletions(-)

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 814ec12..a9bafd3 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
@@ -97,6 +97,18 @@ public class AdvancedFSOps implements IAdvancedFSOps {
 }
 
 /**
+ * Moves a file to a given destination.
+ *
+ * @param fromFile file to move
+ * @param toFile where to move it
+ * @throws IOException on any error
+ */
+@Override
+public void moveFile(File fromFile, File toFile) throws IOException {
+Files.move(fromFile.toPath(), toFile.toPath());
+}
+
+/**
  * Check whether supports atomic directory move.
  * @return true if an atomic directory move works, else false
  */
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 f55ba7f..ccb4a1b 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
@@ -46,6 +46,16 @@ public interface IAdvancedFSOps {
  */
 void moveDirectoryPreferAtomic(File fromDir, File toDir) throws 
IOException;
 
+
+/**
+ * Moves a file to a given destination.
+ *
+ * @param fromFile file to move
+ * @param toFile where to move it
+ * @throws IOException on any error
+ */
+void moveFile(File fromFile, File toFile) throws IOException;
+
 /**
  * Check whether supports atomic directory move.
  * @return true if an atomic directory move works, else false
diff --git 
a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
 
b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
index 3b019f4..c57ebed 100644
--- 
a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
+++ 
b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
@@ -139,9 +139,8 @@ public class LocallyCachedTopologyBlob extends 
LocallyCachedBlob {
 if (isLocalMode && type == TopologyBlobType.TOPO_JAR) {
 LOG.debug("DOWNLOADING LOCAL JAR to TEMP LOCATION... {}", 
topologyId);
 //This is a special case where the jar was not uploaded so we will 
not download it (it is already on the classpath)
-ClassLoader classloader = 
Thread.currentThread().getContextClassLoader();
 String resourcesJar = resourcesJar();
-URL url = 
classloader.getResource(ServerConfigUtils.RESOURCES_SUBDIR);
+URL url = 
ServerUtils.getResourceFromClassloader(ServerConfigUtils.RESOURCES_SUBDIR);
 Path extractionDest = 
topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(LOCAL_MODE_JAR_VERSION));
 if (resourcesJar != null) {
 LOG.info("Extracting resources from jar at {} to {}", 
resourcesJar, extractionDest);
@@ -154,6 +153,10 @@ public class LocallyCachedTopologyBlob extends 
LocallyCachedBlob {
 } else {
 fsOps.copyDirectory(new File(url.getFile()), 
extractionDest.toFile());
 }
+} else if (!fsOps.fileExists(extractionDest)) {
+// if we can't find the resources directory in a resources jar 
or in the classpath just create an empty
+// resources directory. This way we can check later that the 
topology jar was fully downloaded.
+fsOps.forceMkdir(extractionDest);
 }
 return LOCAL_MODE_JAR_VERSION;
 }
@@ -225,7 +228,7 @@ public class LocallyCachedTopologyBlo

[storm] branch master updated: [STORM-3501] local cluster worker restarts because of missing resources folder (#3120)

2019-08-31 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 9c47a02  [STORM-3501] local cluster worker restarts because of missing 
resources folder (#3120)
9c47a02 is described below

commit 9c47a02e52173b55c0184f3d9ecfa4b7d2a79984
Author: Diogo Monteiro 
AuthorDate: Sat Aug 31 18:19:14 2019 +0100

[STORM-3501] local cluster worker restarts because of missing resources 
folder (#3120)

* [STORM-3501] local cluster worker restarts because of missing resources 
folder
---
 .../storm/daemon/supervisor/AdvancedFSOps.java | 12 +++
 .../storm/daemon/supervisor/IAdvancedFSOps.java| 10 +++
 .../storm/localizer/LocallyCachedTopologyBlob.java |  9 +-
 .../java/org/apache/storm/utils/ServerUtils.java   | 14 +++
 .../apache/storm/localizer/AsyncLocalizerTest.java | 99 ++
 5 files changed, 141 insertions(+), 3 deletions(-)

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 23e91bf..d2a9ced 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
@@ -97,6 +97,18 @@ public class AdvancedFSOps implements IAdvancedFSOps {
 }
 
 /**
+ * Moves a file to a given destination.
+ *
+ * @param fromFile file to move
+ * @param toFile where to move it
+ * @throws IOException on any error
+ */
+@Override
+public void moveFile(File fromFile, File toFile) throws IOException {
+Files.move(fromFile.toPath(), toFile.toPath());
+}
+
+/**
  * Check whether supports atomic directory move.
  * @return true if an atomic directory move works, else false
  */
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 f55ba7f..ccb4a1b 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
@@ -46,6 +46,16 @@ public interface IAdvancedFSOps {
  */
 void moveDirectoryPreferAtomic(File fromDir, File toDir) throws 
IOException;
 
+
+/**
+ * Moves a file to a given destination.
+ *
+ * @param fromFile file to move
+ * @param toFile where to move it
+ * @throws IOException on any error
+ */
+void moveFile(File fromFile, File toFile) throws IOException;
+
 /**
  * Check whether supports atomic directory move.
  * @return true if an atomic directory move works, else false
diff --git 
a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
 
b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
index 4ba41f9..babed56 100644
--- 
a/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
+++ 
b/storm-server/src/main/java/org/apache/storm/localizer/LocallyCachedTopologyBlob.java
@@ -139,9 +139,8 @@ public class LocallyCachedTopologyBlob extends 
LocallyCachedBlob {
 if (isLocalMode && type == TopologyBlobType.TOPO_JAR) {
 LOG.debug("DOWNLOADING LOCAL JAR to TEMP LOCATION... {}", 
topologyId);
 //This is a special case where the jar was not uploaded so we will 
not download it (it is already on the classpath)
-ClassLoader classloader = 
Thread.currentThread().getContextClassLoader();
 String resourcesJar = resourcesJar();
-URL url = 
classloader.getResource(ServerConfigUtils.RESOURCES_SUBDIR);
+URL url = 
ServerUtils.getResourceFromClassloader(ServerConfigUtils.RESOURCES_SUBDIR);
 Path extractionDest = 
topologyBasicBlobsRootDir.resolve(type.getTempExtractionDir(LOCAL_MODE_JAR_VERSION));
 if (resourcesJar != null) {
 LOG.info("Extracting resources from jar at {} to {}", 
resourcesJar, extractionDest);
@@ -154,6 +153,10 @@ public class LocallyCachedTopologyBlob extends 
LocallyCachedBlob {
 } else {
 fsOps.copyDirectory(new File(url.getFile()), 
extractionDest.toFile());
 }
+} else if (!fsOps.fileExists(extractionDest)) {
+// if we can't find the resources directory in a resources jar 
or in the classpath just create an empty
+// resources directory. This way we can check later that the 
topology jar was fully downloaded.
+fsOps.forceMkdir(extractionDest);
 }
 return LOCAL_MODE_JAR_VERSION;
 }
@@ -225,7 +228,7 @@ public class LocallyCachedTopologyBlo

[storm] branch 2.1.x-branch updated: STORM-3500: Fix spelling error in property storm.blobstore.dependency.jar.upload.chuck.size.bytes

2019-08-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 2.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/2.1.x-branch by this push:
 new c18e3e3  STORM-3500: Fix spelling error in property 
storm.blobstore.dependency.jar.upload.chuck.size.bytes
c18e3e3 is described below

commit c18e3e3b7aea86ec89d45e1a4c21282f58b3d09c
Author: Stig Rohde Døssing 
AuthorDate: Tue Aug 27 22:28:50 2019 +0200

STORM-3500: Fix spelling error in property 
storm.blobstore.dependency.jar.upload.chuck.size.bytes
---
 conf/defaults.yaml  | 2 +-
 storm-client/src/jvm/org/apache/storm/Config.java   | 6 +++---
 .../src/jvm/org/apache/storm/dependency/DependencyUploader.java | 6 +++---
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 8434b1d..ee5fe56 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -144,7 +144,7 @@ nimbus.blobstore.class: 
"org.apache.storm.blobstore.LocalFsBlobStore"
 nimbus.blobstore.expiration.secs: 600
 
 storm.blobstore.inputstream.buffer.size.bytes: 65536
-storm.blobstore.dependency.jar.upload.chuck.size.bytes: 1048576
+storm.blobstore.dependency.jar.upload.chunk.size.bytes: 1048576
 client.blobstore.class: "org.apache.storm.blobstore.NimbusBlobStore"
 storm.blobstore.replication.factor: 3
 # For secure mode we would want to change this config to true
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java 
b/storm-client/src/jvm/org/apache/storm/Config.java
index a2aa58a..979b287 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -1412,12 +1412,12 @@ public class Config extends HashMap {
 @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.
+ * What chunk 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";
+public static final String 
STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUNK_SIZE_BYTES =
+"storm.blobstore.dependency.jar.upload.chunk.size.bytes";
 /**
  * FQCN of a class that implements {@code ISubmitterHook} @see 
ISubmitterHook for details.
  */
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 1f6b4f4..85ab835 100644
--- a/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
@@ -48,11 +48,11 @@ public class DependencyUploader {
 
 private final Map conf;
 private ClientBlobStore blobStore;
-private int uploadChuckSize;
+private final int uploadChunkSize;
 
 public DependencyUploader() {
 conf = Utils.readStormConfig();
-this.uploadChuckSize = 
ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUCK_SIZE_BYTES),
 1024 * 1024);
+this.uploadChunkSize = 
ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUNK_SIZE_BYTES),
 1024 * 1024);
 }
 
 public void init() {
@@ -164,7 +164,7 @@ public class DependencyUploader {
 try {
 blob = getBlobStore().createBlob(key, new 
SettableBlobMeta(acls));
 try (InputStream in = 
Files.newInputStream(dependency.toPath())) {
-IOUtils.copy(in, blob, this.uploadChuckSize);
+IOUtils.copy(in, blob, this.uploadChunkSize);
 }
 blob.close();
 blob = null;



[storm] branch master updated: STORM-3500: Fix spelling error in property storm.blobstore.dependency.jar.upload.chuck.size.bytes

2019-08-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 405c90b  STORM-3500: Fix spelling error in property 
storm.blobstore.dependency.jar.upload.chuck.size.bytes
 new 2c095ee  Merge pull request #3118 from srdo/STORM-3500
405c90b is described below

commit 405c90ba16bb886ae9727b65682ad902c8f2fb0b
Author: Stig Rohde Døssing 
AuthorDate: Tue Aug 27 22:28:50 2019 +0200

STORM-3500: Fix spelling error in property 
storm.blobstore.dependency.jar.upload.chuck.size.bytes
---
 conf/defaults.yaml  | 2 +-
 storm-client/src/jvm/org/apache/storm/Config.java   | 6 +++---
 .../src/jvm/org/apache/storm/dependency/DependencyUploader.java | 6 +++---
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 1a17e52..e555ac8 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -144,7 +144,7 @@ nimbus.blobstore.class: 
"org.apache.storm.blobstore.LocalFsBlobStore"
 nimbus.blobstore.expiration.secs: 600
 
 storm.blobstore.inputstream.buffer.size.bytes: 65536
-storm.blobstore.dependency.jar.upload.chuck.size.bytes: 1048576
+storm.blobstore.dependency.jar.upload.chunk.size.bytes: 1048576
 client.blobstore.class: "org.apache.storm.blobstore.NimbusBlobStore"
 storm.blobstore.replication.factor: 3
 # For secure mode we would want to change this config to true
diff --git a/storm-client/src/jvm/org/apache/storm/Config.java 
b/storm-client/src/jvm/org/apache/storm/Config.java
index 5d89fc9..b46c112 100644
--- a/storm-client/src/jvm/org/apache/storm/Config.java
+++ b/storm-client/src/jvm/org/apache/storm/Config.java
@@ -1418,12 +1418,12 @@ public class Config extends HashMap {
 @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.
+ * What chunk 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";
+public static final String 
STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUNK_SIZE_BYTES =
+"storm.blobstore.dependency.jar.upload.chunk.size.bytes";
 /**
  * FQCN of a class that implements {@code ISubmitterHook} @see 
ISubmitterHook for details.
  */
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 1f6b4f4..85ab835 100644
--- a/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
+++ b/storm-client/src/jvm/org/apache/storm/dependency/DependencyUploader.java
@@ -48,11 +48,11 @@ public class DependencyUploader {
 
 private final Map conf;
 private ClientBlobStore blobStore;
-private int uploadChuckSize;
+private final int uploadChunkSize;
 
 public DependencyUploader() {
 conf = Utils.readStormConfig();
-this.uploadChuckSize = 
ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUCK_SIZE_BYTES),
 1024 * 1024);
+this.uploadChunkSize = 
ObjectReader.getInt(conf.get(Config.STORM_BLOBSTORE_DEPENDENCY_JAR_UPLOAD_CHUNK_SIZE_BYTES),
 1024 * 1024);
 }
 
 public void init() {
@@ -164,7 +164,7 @@ public class DependencyUploader {
 try {
 blob = getBlobStore().createBlob(key, new 
SettableBlobMeta(acls));
 try (InputStream in = 
Files.newInputStream(dependency.toPath())) {
-IOUtils.copy(in, blob, this.uploadChuckSize);
+IOUtils.copy(in, blob, this.uploadChunkSize);
 }
 blob.close();
 blob = null;



[storm] branch 2.1.x-branch updated: Revert "STORM-3476 don't query remote files on cleanup if target size is acceptable"

2019-08-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 2.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/2.1.x-branch by this push:
 new 4102e33  Revert "STORM-3476 don't query remote files on cleanup if 
target size is acceptable"
4102e33 is described below

commit 4102e33930e32fa312842ae51ab3d98ed8becb77
Author: Stig Rohde Døssing 
AuthorDate: Tue Aug 27 22:29:35 2019 +0200

Revert "STORM-3476 don't query remote files on cleanup if target size is 
acceptable"

This reverts commit 1008feba371dd33232070603a54f8b3f4ed49ae7.
---
 .../org/apache/storm/localizer/LocalizedResourceRetentionSet.java | 4 
 1 file changed, 4 deletions(-)

diff --git 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
index 25d6f4d..f96b481 100644
--- 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
+++ 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
@@ -81,10 +81,6 @@ public class LocalizedResourceRetentionSet {
 public void cleanup(ClientBlobStore store) {
 LOG.debug("cleanup target size: {} current size is: {}", targetSize, 
currentSize);
 long bytesOver = currentSize - targetSize;
-if (bytesOver <= 0) { // no need to query remote files
-return;
-}
-
 //First delete everything that no longer exists...
 for (Iterator>> i = noReferences.entrySet().iterator();
  i.hasNext(); ) {



[storm] branch master updated: Revert "STORM-3476 don't query remote files on cleanup if target size is acceptable"

2019-08-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 3b993a9  Revert "STORM-3476 don't query remote files on cleanup if 
target size is acceptable"
 new b27da9f  Merge pull request #3119 from srdo/revert-STORM-3476
3b993a9 is described below

commit 3b993a98ea42bec22d8ad7eb9699f30c6afba9b7
Author: Stig Rohde Døssing 
AuthorDate: Tue Aug 27 22:29:35 2019 +0200

Revert "STORM-3476 don't query remote files on cleanup if target size is 
acceptable"

This reverts commit 1008feba371dd33232070603a54f8b3f4ed49ae7.
---
 .../org/apache/storm/localizer/LocalizedResourceRetentionSet.java | 4 
 1 file changed, 4 deletions(-)

diff --git 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
index ddb59c8..3c31d4e 100644
--- 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
+++ 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
@@ -81,10 +81,6 @@ public class LocalizedResourceRetentionSet {
 public void cleanup(ClientBlobStore store) {
 LOG.debug("cleanup target size: {} current size is: {}", targetSize, 
currentSize);
 long bytesOver = currentSize - targetSize;
-if (bytesOver <= 0) { // no need to query remote files
-return;
-}
-
 //First delete everything that no longer exists...
 for (Iterator>> i = noReferences.entrySet().iterator();
  i.hasNext(); ) {



[storm] branch master updated: STORM-3437: Automate license checks more (#3053)

2019-08-14 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 6bc2a85  STORM-3437: Automate license checks more (#3053)
6bc2a85 is described below

commit 6bc2a858ead4d5bf9cdf3edaa881ad7d682d139a
Author: Stig Døssing 
AuthorDate: Wed Aug 14 16:21:11 2019 +0200

STORM-3437: Automate license checks more (#3053)

STORM-3437: Automate license checks more, add CI job to check that license 
files are up to date
---
 .travis.yml |   7 +
 DEPENDENCY-LICENSES | 258 +++-
 DEVELOPER.md|   8 +-
 LICENSE-binary  | 184 +
 dev-tools/list_jars.sh  |  41 --
 dev-tools/travis/travis-script.sh   |   3 +
 dev-tools/validate-license-files.py | 175 
 pom.xml | 121 +
 storm-dist/binary/pom.xml   |   2 +
 9 files changed, 417 insertions(+), 382 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 1a79f48..d35d5e3 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -19,6 +19,7 @@ env:
   - MODULES=Core
   - MODULES=External
   - MODULES=Integration-Test
+  - MODULES=Check-Updated-License-Files
 
 dist: trusty
 sudo: required
@@ -39,6 +40,12 @@ before_install:
   - sudo apt-get update
   - sudo apt-get install python3
   - sudo apt-get install python3-pip
+  - sudo add-apt-repository ppa:deadsnakes/ppa -y
+  - sudo apt-get update
+  - sudo apt-get install python3.6
+  - wget 
http://mirrors.rackhosting.com/apache/maven/maven-3/3.6.1/binaries/apache-maven-3.6.1-bin.tar.gz
 -P $HOME
+  - tar xzvf $HOME/apache-maven-3.6.1-bin.tar.gz -C $HOME
+  - export PATH=$HOME/apache-maven-3.6.1/bin:$PATH
 install: /bin/bash ./dev-tools/travis/travis-install.sh `pwd`
 script:
   - /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES
diff --git a/DEPENDENCY-LICENSES b/DEPENDENCY-LICENSES
index 16cc35a..3761367 100644
--- a/DEPENDENCY-LICENSES
+++ b/DEPENDENCY-LICENSES
@@ -5,11 +5,7 @@ List of third-party dependencies grouped by their license type.
 Apache License
 
 * HttpClient (commons-httpclient:commons-httpclient:3.0.1 - 
http://jakarta.apache.org/commons/httpclient/)
-* HttpClient (commons-httpclient:commons-httpclient:3.1 - 
http://jakarta.apache.org/httpcomponents/httpclient-3.x/)
-* HttpClient (org.apache.httpcomponents:httpclient:4.2.5 - 
http://hc.apache.org/httpcomponents-client)
-* HttpCore (org.apache.httpcomponents:httpcore:4.2.4 - 
http://hc.apache.org/httpcomponents-core-ga)
-* Log4j Implemented Over SLF4J (org.slf4j:log4j-over-slf4j:1.6.6 - 
http://www.slf4j.org)
-* Log4j Implemented Over SLF4J (org.slf4j:log4j-over-slf4j:1.7.25 - 
http://www.slf4j.org)
+* Log4j Implemented Over SLF4J (org.slf4j:log4j-over-slf4j:1.7.26 - 
http://www.slf4j.org)
 
 Apache License, Version 2.0
 
@@ -37,19 +33,14 @@ List of third-party dependencies grouped by their license 
type.
 * Apache Calcite Avatica 
(org.apache.calcite.avatica:avatica-core:1.10.0 - 
https://calcite.apache.org/avatica/avatica-core)
 * Apache Calcite Avatica Metrics 
(org.apache.calcite.avatica:avatica-metrics:1.10.0 - 
https://calcite.apache.org/avatica/avatica-metrics)
 * Apache Calcite Avatica Metrics 
(org.apache.calcite.avatica:avatica-metrics:1.8.0 - 
http://calcite.apache.org/avatica/avatica-metrics)
-* Apache Commons CLI (commons-cli:commons-cli:1.3.1 - 
http://commons.apache.org/proper/commons-cli/)
 * Apache Commons CLI (commons-cli:commons-cli:1.4 - 
http://commons.apache.org/proper/commons-cli/)
-* Apache Commons Codec (commons-codec:commons-codec:1.10 - 
http://commons.apache.org/proper/commons-codec/)
 * Apache Commons Codec (commons-codec:commons-codec:1.11 - 
http://commons.apache.org/proper/commons-codec/)
-* Apache Commons Codec (commons-codec:commons-codec:1.9 - 
http://commons.apache.org/proper/commons-codec/)
 * Apache Commons Collections 
(commons-collections:commons-collections:3.2.2 - 
http://commons.apache.org/collections/)
 * Apache Commons Compress (org.apache.commons:commons-compress:1.18 - 
https://commons.apache.org/proper/commons-compress/)
-* Apache Commons Compress (org.apache.commons:commons-compress:1.9 - 
http://commons.apache.org/proper/commons-compress/)
 * Apache Commons Crypto (org.apache.commons:commons-crypto:1.0.0 - 
http://commons.apache.org/proper/commons-crypto/)
 * Apache Commons CSV (org.apache.commons:commons-csv:1.4 - 
http://commons.apache.org/proper/commons-csv/)
 * Apache Commons Exec (org.apache.commons:commons-exec:1.3 - 
http://commons.apache.org/proper/commons-exec/)
 * Apache Commons FileUpload 
(commons

[storm] branch master updated: STORM-3372: Fix NPE when shutting down HdfsBolt, fix storm-hdfs tests not running

2019-07-29 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new d1912ae  STORM-3372: Fix NPE when shutting down HdfsBolt, fix 
storm-hdfs tests not running
 new f015130  Merge pull request #2990 from srdo/STORM-3372
d1912ae is described below

commit d1912ae98afe9f470a05e57835c41f056cebb311
Author: Stig Rohde Døssing 
AuthorDate: Thu Apr 4 15:58:55 2019 +0200

STORM-3372: Fix NPE when shutting down HdfsBolt, fix storm-hdfs tests not 
running
---
 external/storm-hdfs-blobstore/pom.xml  |   5 +
 .../apache/storm/hdfs/blobstore/BlobStoreTest.java | 266 ++---
 external/storm-hdfs/pom.xml|   2 +-
 .../apache/storm/hdfs/bolt/AbstractHdfsBolt.java   |   4 +-
 .../org/apache/storm/hdfs/bolt/TestHdfsBolt.java   |  12 +
 .../hdfs/bolt/format/TestSimpleFileNameFormat.java |   2 +-
 .../apache/storm/hdfs/spout/TestHdfsSemantics.java |   5 +-
 .../org/apache/storm/hdfs/spout/TestHdfsSpout.java |  25 +-
 .../hdfs/testing/MiniDFSClusterExtension.java  |  64 +
 .../storm/hdfs/testing/MiniDFSClusterRule.java |   5 +
 .../storm-hdfs/src/test/resources/log4j.properties |   5 +-
 external/storm-hdfs/src/test/resources/log4j2.xml  |  32 +++
 pom.xml|   2 +-
 13 files changed, 268 insertions(+), 161 deletions(-)

diff --git a/external/storm-hdfs-blobstore/pom.xml 
b/external/storm-hdfs-blobstore/pom.xml
index e2b948b..ccc69a6 100644
--- a/external/storm-hdfs-blobstore/pom.xml
+++ b/external/storm-hdfs-blobstore/pom.xml
@@ -208,6 +208,11 @@
 guava
 ${guava.version}
 
+
+org.junit.jupiter
+junit-jupiter-params
+test
+
 
 
 
diff --git 
a/external/storm-hdfs-blobstore/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
 
b/external/storm-hdfs-blobstore/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
index a125793..53cca75 100644
--- 
a/external/storm-hdfs-blobstore/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
+++ 
b/external/storm-hdfs-blobstore/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.storm.hdfs.blobstore;
 
+import org.apache.storm.hdfs.testing.MiniDFSClusterExtension;
 import org.apache.commons.io.FileUtils;
 import org.apache.storm.Config;
 import org.apache.storm.blobstore.AtomicOutputStream;
@@ -28,14 +29,9 @@ import org.apache.storm.generated.AccessControlType;
 import org.apache.storm.generated.AuthorizationException;
 import org.apache.storm.generated.KeyNotFoundException;
 import org.apache.storm.generated.SettableBlobMeta;
-import org.apache.storm.hdfs.testing.MiniDFSClusterRule;
 import org.apache.storm.security.auth.FixedGroupsMapping;
 import org.apache.storm.security.auth.NimbusPrincipal;
 import org.apache.storm.security.auth.SingleUserPrincipal;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,30 +51,34 @@ import java.util.UUID;
 
 import static org.junit.Assert.*;
 
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
 public class BlobStoreTest {
 
-@ClassRule
-public static final MiniDFSClusterRule DFS_CLUSTER_RULE = new 
MiniDFSClusterRule();
+@RegisterExtension
+public static final MiniDFSClusterExtension DFS_CLUSTER_EXTENSION = new 
MiniDFSClusterExtension();
 
 private static final Logger LOG = 
LoggerFactory.getLogger(BlobStoreTest.class);
 URI base;
-File baseFile;
 private static final Map CONF = new HashMap<>();
 public static final int READ = 0x01;
 public static final int WRITE = 0x02;
 public static final int ADMIN = 0x04;
 
-@Before
+@BeforeEach
 public void init() {
 initializeConfigs();
-baseFile = new File("/tmp/blob-store-test-" + UUID.randomUUID());
-base = baseFile.toURI();
 }
 
-@After
+@AfterEach
 public void cleanup()
 throws IOException {
-FileUtils.deleteDirectory(baseFile);
 }
 
 // Method which initializes nimbus admin
@@ -160,7 +160,7 @@ public class BlobStoreTest {
 conf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN, 
"org.apache.storm.security.auth.DefaultPrincipalToLocal");
 conf.put(Config.STORM_BLOBSTORE_REPLICATION_FACTOR, 3);
 HdfsBlobStore store = new HdfsBlobStore();
-store.

[storm] branch master updated: STORM-3476 don't query remote files on cleanup if target size is acceptable

2019-07-29 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 1008feb  STORM-3476 don't query remote files on cleanup if target size 
is acceptable
 new 94e5ad1  Merge pull request #3093 from agresch/agresch_storm_3476
1008feb is described below

commit 1008feba371dd33232070603a54f8b3f4ed49ae7
Author: Aaron Gresch 
AuthorDate: Thu Jul 25 16:29:04 2019 -0500

STORM-3476 don't query remote files on cleanup if target size is acceptable
---
 .../org/apache/storm/localizer/LocalizedResourceRetentionSet.java | 4 
 1 file changed, 4 insertions(+)

diff --git 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
index f96b481..25d6f4d 100644
--- 
a/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
+++ 
b/storm-server/src/main/java/org/apache/storm/localizer/LocalizedResourceRetentionSet.java
@@ -81,6 +81,10 @@ public class LocalizedResourceRetentionSet {
 public void cleanup(ClientBlobStore store) {
 LOG.debug("cleanup target size: {} current size is: {}", targetSize, 
currentSize);
 long bytesOver = currentSize - targetSize;
+if (bytesOver <= 0) { // no need to query remote files
+return;
+}
+
 //First delete everything that no longer exists...
 for (Iterator>> i = noReferences.entrySet().iterator();
  i.hasNext(); ) {



[storm] branch master updated: MINOR: Clarify error message when TGT has problems (#3012)

2019-07-26 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 9f10f8a  MINOR: Clarify error message when TGT has problems (#3012)
9f10f8a is described below

commit 9f10f8a14482e4eb1c1323ac86d3c373634539c2
Author: jacobtolar 
AuthorDate: Fri Jul 26 11:49:05 2019 -0500

MINOR: Clarify error message when TGT has problems (#3012)

If you're not very familiar with Kerberos, having more explicit 
instructions in these error messages can be helpful.
---
 .../src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java| 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

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 e93b1c4..c4f9f91 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
@@ -137,15 +137,15 @@ public class AutoTGT implements IAutoCredentials, 
ICredentialsRenewer, IMetricsR
 }
 
 if (!tgt.isForwardable()) {
-throw new RuntimeException("The TGT found is not 
forwardable. Please use -f option.");
+throw new RuntimeException("The TGT found is not 
forwardable. Please use -f option with 'kinit'.");
 }
 
 if (!tgt.isRenewable()) {
-throw new RuntimeException("The TGT found is not 
renewable. Please use -r option.");
+throw new RuntimeException("The TGT found is not 
renewable. Please use -r option with 'kinit'.");
 }
 
 if (tgt.getClientAddresses() != null) {
-throw new RuntimeException("The TGT found is not 
address-less. Please use -A option.");
+throw new RuntimeException("The TGT found is not 
address-less. Please use -A option with 'kinit'.");
 }
 
 LOG.info("Pushing TGT for " + tgt.getClient() + " to 
topology.");



[storm] branch master updated: STORM-3472: Add tests missing for STORM-3411, make the download file name generat… (#3091)

2019-07-26 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new a694d87  STORM-3472: Add tests missing for STORM-3411, make the 
download file name generat… (#3091)
a694d87 is described below

commit a694d873f93616474430a337d7819e174f9e9b5b
Author: Stig Døssing 
AuthorDate: Fri Jul 26 18:40:29 2019 +0200

STORM-3472: Add tests missing for STORM-3411, make the download file name 
generat… (#3091)

* STORM-3472: Add tests missing for STORM-3411, make the download file name 
generation code less error prone
---
 .../daemon/logviewer/handler/LogviewerProfileHandler.java | 10 ++
 .../storm/daemon/logviewer/utils/LogFileDownloader.java   | 13 -
 .../daemon/logviewer/utils/LogviewerResponseBuilder.java  | 15 ---
 .../storm/daemon/logviewer/webapp/LogviewerResource.java  |  7 +++
 .../handler/LogviewerLogDownloadHandlerTest.java  |  9 -
 .../logviewer/handler/LogviewerProfileHandlerTest.java| 15 +--
 6 files changed, 42 insertions(+), 27 deletions(-)

diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
index 58b883a..c68ab3c 100644
--- 
a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
+++ 
b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerProfileHandler.java
@@ -102,7 +102,6 @@ public class LogviewerProfileHandler {
 /**
  * Download a dump file.
  *
- * @param host host address
  * @param topologyId topology ID
  * @param hostPort host and port of worker
  * @param fileName dump file name
@@ -110,8 +109,10 @@ public class LogviewerProfileHandler {
  * @return a Response which lets browsers download that file.
  * @see {@link 
org.apache.storm.daemon.logviewer.utils.LogFileDownloader#downloadFile(String, 
String, String, boolean)}
  */
-public Response downloadDumpFile(String host, String topologyId, String 
hostPort, String fileName, String user) throws IOException {
-String portStr = hostPort.split(":")[1];
+public Response downloadDumpFile(String topologyId, String hostPort, 
String fileName, String user) throws IOException {
+String[] hostPortSplit = hostPort.split(":");
+String host = hostPortSplit[0];
+String portStr = hostPortSplit[1];
 Path rawFile = 
logRoot.resolve(topologyId).resolve(portStr).resolve(fileName);
 Path absFile = rawFile.toAbsolutePath().normalize();
 if (!absFile.startsWith(logRoot) || 
!rawFile.normalize().toString().equals(rawFile.toString())) {
@@ -122,7 +123,8 @@ public class LogviewerProfileHandler {
 if (absFile.toFile().exists()) {
 String workerFileRelativePath = String.join(File.separator, 
topologyId, portStr, WORKER_LOG_FILENAME);
 if (resourceAuthorizer.isUserAllowedToAccessFile(user, 
workerFileRelativePath)) {
-return LogviewerResponseBuilder.buildDownloadFile(host, 
absFile.toFile(), numFileDownloadExceptions);
+String downloadedFileName = host + "-" + topologyId + "-" + 
portStr + "-" + absFile.getFileName();
+return 
LogviewerResponseBuilder.buildDownloadFile(downloadedFileName, 
absFile.toFile(), numFileDownloadExceptions);
 } else {
 return 
LogviewerResponseBuilder.buildResponseUnauthorizedUser(user);
 }
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
index 17eb10f..22ce298 100644
--- 
a/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
+++ 
b/storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/utils/LogFileDownloader.java
@@ -80,7 +80,18 @@ public class LogFileDownloader {
 if (file.toFile().exists()) {
 if (isDaemon || resourceAuthorizer.isUserAllowedToAccessFile(user, 
fileName)) {
 fileDownloadSizeDistMb.update(Math.round((double) 
file.toFile().length() / FileUtils.ONE_MB));
-return LogviewerResponseBuilder.buildDownloadFile(host, 
file.toFile(), numFileDownloadExceptions);
+String downloadedFileName;
+Path pathRelativeToRootDir = rootDir.relativize(file);
+if (isDaemon || pathRelativeToRootDir.getNameCount() != 3) {
+downloadedFileName = host + "-" + rawFile.getFileName();
+} else {
+//host-topoId-

[storm] branch master updated: STORM-3435: Use Jetty BOM

2019-07-25 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 2c5c9b7  STORM-3435: Use Jetty BOM
 new 2b669f8  Merge pull request #3051 from srdo/STORM-3435
2c5c9b7 is described below

commit 2c5c9b79e0550e7b8d2f8d8315edc2dc3d445ce2
Author: Stig Rohde Døssing 
AuthorDate: Sat Jun 29 12:08:46 2019 +0200

STORM-3435: Use Jetty BOM
---
 pom.xml | 20 +++-
 1 file changed, 3 insertions(+), 17 deletions(-)

diff --git a/pom.xml b/pom.xml
index 5603831..6ad8c43 100644
--- a/pom.xml
+++ b/pom.xml
@@ -865,25 +865,11 @@
 
 
 org.eclipse.jetty
-jetty-servlet
-${jetty.version}
-
-
-org.eclipse.jetty
-jetty-servlets
-${jetty.version}
-
-
-org.eclipse.jetty
-jetty-server
-${jetty.version}
-
-
-org.eclipse.jetty
-jetty-util
+jetty-bom
 ${jetty.version}
+import
+pom
 
-
 
 org.clojure
 tools.logging



[storm-site] 02/02: Rebuild site

2019-07-19 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit 1f0067b65e0d3ec47904d06ce80bb34fb8fe4ba3
Author: Stig Rohde Døssing 
AuthorDate: Fri Jul 19 19:11:15 2019 +0200

Rebuild site
---
 content/2019/05/30/storm200-released.html | 2 +-
 content/feed.xml  | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/content/2019/05/30/storm200-released.html 
b/content/2019/05/30/storm200-released.html
index de3c77b..0f5c4d6 100644
--- a/content/2019/05/30/storm200-released.html
+++ b/content/2019/05/30/storm200-released.html
@@ -262,7 +262,7 @@
 
 The most significant change to Storms Kafka integration since 1.x, is 
that storm-kafka has been removed. The module was deprecated a while back, due 
to Kafkas deprecation of the underlying client library. Users will have to 
move to the storm-kafka-client module, which uses Kafkas ´kafka-clients´ 
library for integration.
 
-For the most part, the migration to storm-kafka-client is straightforward. 
The documentation for storm-kafka-client contains a helpful mapping between the 
old and new spout configurations. If you are using any of the storm-kafka 
spouts, you will need to migrate offset checkpoints to the new spout, to avoid 
the new spout starting from scratch on your partitions. Storm provides a helper 
tool to do this which can be found https://github.com/apache/storm/tree/master/external/sto [...]
+For the most part, the migration to storm-kafka-client is straightforward. 
The documentation for storm-kafka-client contains a helpful mapping between the 
old and new spout configurations. If you are using any of the storm-kafka 
spouts, you will need to migrate offset checkpoints to the new spout, to avoid 
the new spout starting from scratch on your partitions. Storm provides a helper 
tool to do this which can be found https://github.com/apache/storm/tree/master/external/stor [...]
 
 When performing a migration, you should stop your topology, run the 
migration tool, then redeploy your topology with the storm-kafka-client 
spout.
 
diff --git a/content/feed.xml b/content/feed.xml
index 440c1c4..9e2ea3e 100644
--- a/content/feed.xml
+++ b/content/feed.xml
@@ -5,8 +5,8 @@
 
 http://storm.apache.org/
 http://storm.apache.org/feed.xml; rel="self" 
type="application/rss+xml"/>
-Thu, 18 Jul 2019 22:29:01 +0200
-Thu, 18 Jul 2019 22:29:01 +0200
+Fri, 19 Jul 2019 19:07:38 +0200
+Fri, 19 Jul 2019 19:07:38 +0200
 Jekyll v3.6.2
 
   
@@ -121,7 +121,7 @@
 
 pThe most significant change to Storm#39;s Kafka integration 
since 1.x, is that storm-kafka has been removed. The module was deprecated a 
while back, due to Kafka#39;s deprecation of the underlying client 
library. Users will have to move to the storm-kafka-client module, which uses 
Kafka#39;s ´kafka-clients´ library for integration./p
 
-pFor the most part, the migration to storm-kafka-client is 
straightforward. The documentation for storm-kafka-client contains a helpful 
mapping between the old and new spout configurations. If you are using any of 
the storm-kafka spouts, you will need to migrate offset checkpoints to the new 
spout, to avoid the new spout starting from scratch on your partitions. Storm 
provides a helper tool to do this which can be found a 
href=fhttps://github.com/apache/storm/tree/maste [...]
+pFor the most part, the migration to storm-kafka-client is 
straightforward. The documentation for storm-kafka-client contains a helpful 
mapping between the old and new spout configurations. If you are using any of 
the storm-kafka spouts, you will need to migrate offset checkpoints to the new 
spout, to avoid the new spout starting from scratch on your partitions. Storm 
provides a helper tool to do this which can be found a 
href=https://github.com/apache/storm/tree/master [...]
 
 pWhen performing a migration, you should stop your topology, run the 
migration tool, then redeploy your topology with the storm-kafka-client 
spout./p
 



[storm-site] 01/02: Fix dead link in 2.0.0 announcement

2019-07-19 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit ce7b80606743af87334ad99bf36244675208bbed
Author: Stig Rohde Døssing 
AuthorDate: Fri Jul 19 19:07:33 2019 +0200

Fix dead link in 2.0.0 announcement
---
 _posts/2019-05-30-storm200-released.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/_posts/2019-05-30-storm200-released.md 
b/_posts/2019-05-30-storm200-released.md
index 38f1264..abbb110 100644
--- a/_posts/2019-05-30-storm200-released.md
+++ b/_posts/2019-05-30-storm200-released.md
@@ -34,7 +34,7 @@ For more details see [stateful windowing 
documentation](https://github.com/apach
 ### Removal of Storm-Kafka
 The most significant change to Storm's Kafka integration since 1.x, is that 
storm-kafka has been removed. The module was deprecated a while back, due to 
Kafka's deprecation of the underlying client library. Users will have to move 
to the storm-kafka-client module, which uses Kafka's ´kafka-clients´ library 
for integration.
 
-For the most part, the migration to storm-kafka-client is straightforward. The 
documentation for storm-kafka-client contains a helpful mapping between the old 
and new spout configurations. If you are using any of the storm-kafka spouts, 
you will need to migrate offset checkpoints to the new spout, to avoid the new 
spout starting from scratch on your partitions. Storm provides a helper tool to 
do this which can be found 
[here](fhttps://github.com/apache/storm/tree/master/external/storm-ka [...]
+For the most part, the migration to storm-kafka-client is straightforward. The 
documentation for storm-kafka-client contains a helpful mapping between the old 
and new spout configurations. If you are using any of the storm-kafka spouts, 
you will need to migrate offset checkpoints to the new spout, to avoid the new 
spout starting from scratch on your partitions. Storm provides a helper tool to 
do this which can be found 
[here](https://github.com/apache/storm/tree/master/external/storm-kaf [...]
 
 When performing a migration, you should stop your topology, run the migration 
tool, then redeploy your topology with the storm-kafka-client spout.
 



[storm-site] branch asf-site updated (e6f8b2b -> 1f0067b)

2019-07-19 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git.


from e6f8b2b  Regenerate site
 new ce7b806  Fix dead link in 2.0.0 announcement
 new 1f0067b  Rebuild site

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 _posts/2019-05-30-storm200-released.md| 2 +-
 content/2019/05/30/storm200-released.html | 2 +-
 content/feed.xml  | 6 +++---
 3 files changed, 5 insertions(+), 5 deletions(-)



svn commit: r34923 - in /release/storm: apache-storm-1.0.6/ apache-storm-1.1.2/ apache-storm-1.1.3/ apache-storm-1.2.0/ apache-storm-1.2.1/ apache-storm-1.2.2/

2019-07-18 Thread srdo
Author: srdo
Date: Thu Jul 18 20:40:08 2019
New Revision: 34923

Log:
Delete old Storm releases

Removed:
release/storm/apache-storm-1.0.6/
release/storm/apache-storm-1.1.2/
release/storm/apache-storm-1.1.3/
release/storm/apache-storm-1.2.0/
release/storm/apache-storm-1.2.1/
release/storm/apache-storm-1.2.2/



[storm-site] 01/03: Add 1.2.3 to downloads page, remove all old releases from downloads page, clean up releases plugin a bit, delete releases.html as we don't link to it anywhere

2019-07-18 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit bb9f386abc2f384a6942d6849547a842f1080fad
Author: Stig Rohde Døssing 
AuthorDate: Thu Jul 18 21:52:54 2019 +0200

Add 1.2.3 to downloads page, remove all old releases from downloads page, 
clean up releases plugin a bit, delete releases.html as we don't link to it 
anywhere
---
 _data/releases.yml   |  18 ---
 _plugins/releases.rb |  34 +-
 downloads.html   | 335 +++
 releases.html|  44 ---
 4 files changed, 22 insertions(+), 409 deletions(-)

diff --git a/_data/releases.yml b/_data/releases.yml
deleted file mode 100644
index aed64fd..000
--- a/_data/releases.yml
+++ /dev/null
@@ -1,18 +0,0 @@
-- name: 1.0.0
-- name: 1.0.1
-- name: 1.0.2
-- name: 1.0.3-SNAPSHOT
-  git-tag-or-branch: 1.0.x-branch
-- name: 1.1.0-SNAPSHOT
-  git-tag-or-branch: 1.x-branch
-- name: 2.0.0-SNAPSHOT
-  git-tag-or-branch: master
-- name: 0.10.1
-- name: 0.10.0
-- name: 0.10.0-beta1
-- name: 0.9.6
-- name: 0.9.5
-- name: 0.9.4
-- name: 0.9.3
-- name: 0.9.2-incubating
-- name: 0.9.1-incubating
diff --git a/_plugins/releases.rb b/_plugins/releases.rb
index f28ccd2..8f0d4e0 100644
--- a/_plugins/releases.rb
+++ b/_plugins/releases.rb
@@ -20,46 +20,19 @@ module Releases
   return version_string.split('.').map{|e| e.to_i}
 end
 
-def release_from_pom()
-  text= `mvn -f ../pom.xml help:evaluate -Dexpression=project.version`
-  return text.split("\n").select{|a| !a.start_with?('[')}[0]
-end
-
-def branch_from_git()
-  return `git rev-parse --abbrev-ref HEAD`
-end
-
 def generate(site)
-  if site.config['storm_release_only']
-release_name = release_from_pom()
-puts "release: #{release_name}"
-git_branch = branch_from_git()
-puts "branch: #{git_branch}"
-for page in site.pages do
-  page.data['version'] = release_name;
-  page.data['git-tree-base'] = 
"http://github.com/apache/storm/tree/#{git_branch};
-  page.data['git-blob-base'] = 
"http://github.com/apache/storm/blob/#{git_branch};
-end
-return
-  end
-
   releases = Hash.new
-  if (site.data['releases'])
-for rel_data in site.data['releases'] do
-  releases[rel_data['name']] = rel_data
-end
-  end
 
+  # Find the releases/ subdirectories, their names are the current releases
   for page in site.pages do
 release_name = dir_to_releasename(page.dir)
 if (release_name != nil)
-  if !releases.has_key?(release_name)
-releases[release_name] = {'name' => release_name};
-  end
+  releases[release_name] = {'name' => release_name};
   releases[release_name]['documented'] = true
 end
   end
 
+  # Set some metadata for each release
   releases.each { |release_name, release_data|
   set_if_unset(release_data, 'git-tag-or-branch', 
"v#{release_data['name']}")
   set_if_unset(release_data, 'git-tree-base', 
"http://github.com/apache/storm/tree/#{release_data['git-tag-or-branch']}")
@@ -68,6 +41,7 @@ module Releases
   set_if_unset(release_data, 'has-download', 
!release_name.end_with?('-SNAPSHOT'))
   }
 
+  # Make release metadata available to each page for release specific 
docs, e.g. links to source
   for page in site.pages do
 release_name = dir_to_releasename(page.dir)
 if (release_name != nil)
diff --git a/downloads.html b/downloads.html
index 8314afd..02c752d 100644
--- a/downloads.html
+++ b/downloads.html
@@ -60,29 +60,29 @@ artifactId: storm-{component}
 version: 2.0.0 
  
  Current 1.2.x Release
- The current 1.2.x release is 1.2.2. Source 
and binary distributions can be found below.
+ The current 1.2.x release is 1.2.3. Source 
and binary distributions can be found below.
  
- The list of changes for this release can be 
found https://www.apache.org/dist/storm/apache-storm-1.2.2/RELEASE_NOTES.html;>here.
+ The list of changes for this release can be 
found https://www.apache.org/dist/storm/apache-storm-1.2.3/RELEASE_NOTES.html;>here.
   
-  Documentation
-  Javadocs
+  Documentation
+  Javadocs
 
  
- https://www.apache.org/dyn/closer.lua/storm/apache-storm-1.2.2/apache-storm-1.2.2.tar.gz;>apache-storm-1.2.2.tar.gz
- 

[storm] branch master updated: MINOR: Add info about removed JavadocMethod check to storm_checkstyle.xml comment (#3086)

2019-07-14 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new e1dcb03  MINOR: Add info about removed JavadocMethod check to 
storm_checkstyle.xml comment (#3086)
e1dcb03 is described below

commit e1dcb03e22f78ac6eb853e817b1067c09235c6f7
Author: Karl-Philipp Richter 
AuthorDate: Sun Jul 14 12:02:34 2019 +0200

MINOR: Add info about removed JavadocMethod check to storm_checkstyle.xml 
comment (#3086)
---
 storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml | 1 +
 1 file changed, 1 insertion(+)

diff --git a/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml 
b/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
index d1497c8..4b20a6f 100644
--- a/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
+++ b/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
@@ -27,6 +27,7 @@
 It has been slightly modified for use in Apache Storm, as follows:
   * 4 space indents instead of 2
   * line-length limit is 140 instead of 100
+  * removed JavadocMethod
 Once checkstyle has the ability to override selected configuration 
elements from within the Maven
 pom.xml file, then we can remove this file in favor of overriding the 
provided google_checks.xml file.
 See this issue to track that functionality:



[storm] branch master updated: Remove unused dev-tools/test-ns.py

2019-07-14 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 2ac0a0d  Remove unused dev-tools/test-ns.py
 new bb680dd  Merge pull request #3085 from krichter722/missing-batch-mode
2ac0a0d is described below

commit 2ac0a0d09fb778dfd74f92fabffe73af4bd24115
Author: Karl-Philipp Richter 
AuthorDate: Sat Jun 1 17:55:11 2019 +0200

Remove unused dev-tools/test-ns.py
---
 dev-tools/test-ns.py | 30 --
 1 file changed, 30 deletions(-)

diff --git a/dev-tools/test-ns.py b/dev-tools/test-ns.py
deleted file mode 100755
index c0749e8..000
--- a/dev-tools/test-ns.py
+++ /dev/null
@@ -1,30 +0,0 @@
-#!/usr/bin/env python
-# 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.
-
-from subprocess import Popen, PIPE
-import sys
-import os
-
-os.chdir("storm-core")
-
-ns = sys.argv[1]
-pipe = Popen(["mvn", "test", "-DfailIfNoTests=false", "-Dtest=%s"%ns])
-pipe.wait()
-
-os.chdir("..")
-
-sys.exit(pipe.returncode)



[storm] branch master updated: STORM-3422: Make the TupleCaptureBolt thread-safe

2019-07-08 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 60a93ad  STORM-3422: Make the TupleCaptureBolt thread-safe
 new 727caf6  Merge pull request #3034 from 
JanecekPetr/3422-TupleCaptureBolt_is_not_thread-safe
60a93ad is described below

commit 60a93add93bc4e1d63ad6d3356a81626ace00920
Author: Petr Janeček 
AuthorDate: Mon Jul 8 19:06:32 2019 +0200

STORM-3422: Make the TupleCaptureBolt thread-safe
---
 .../org/apache/storm/testing/TupleCaptureBolt.java | 33 ++
 1 file changed, 22 insertions(+), 11 deletions(-)

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 19bdf86..6d07372 100644
--- a/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java
+++ b/storm-client/src/jvm/org/apache/storm/testing/TupleCaptureBolt.java
@@ -17,22 +17,29 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.tuple.Tuple;
 
-
 public class TupleCaptureBolt implements IRichBolt {
-public static final transient Map>> 
emitted_tuples = new HashMap<>();
 
-private String name;
+/*
+ * Even though normally bolts do not need to care about thread safety, 
this particular bolt is different.
+ * It maintains a static field that is prepopulated before the topology 
starts, is written into by the topology,
+ * and is then read from after the topology is completed - all of this by 
potentially different threads.
+ */
+
+private static final transient Map>> 
emitted_tuples = new ConcurrentHashMap<>();
+
+private final String name;
 private OutputCollector collector;
 
 public TupleCaptureBolt() {
 name = UUID.randomUUID().toString();
-emitted_tuples.put(name, new HashMap>());
+emitted_tuples.put(name, new ConcurrentHashMap>());
 }
 
 @Override
@@ -43,11 +50,14 @@ public class TupleCaptureBolt implements IRichBolt {
 @Override
 public void execute(Tuple input) {
 String component = input.getSourceComponent();
-Map> captured = emitted_tuples.get(name);
-if (!captured.containsKey(component)) {
-captured.put(component, new ArrayList());
-}
-captured.get(component).add(new FixedTuple(input.getSourceStreamId(), 
input.getValues()));
+emitted_tuples.get(name)
+.compute(component, (String key, List tuples) -> {
+if (tuples == null) {
+tuples = new ArrayList<>();
+}
+tuples.add(new FixedTuple(input.getSourceStreamId(), 
input.getValues()));
+return tuples;
+});
 collector.ack(input);
 }
 
@@ -64,8 +74,9 @@ public class TupleCaptureBolt implements IRichBolt {
 }
 
 public Map> getAndClearResults() {
-Map> ret = new 
HashMap<>(emitted_tuples.get(name));
-emitted_tuples.get(name).clear();
+Map> results = emitted_tuples.get(name);
+Map> ret = new HashMap<>(results);
+results.clear();
 return ret;
 }
 



[storm] branch master updated: MINOR: Fix powered by readme link

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 65ab12c  MINOR: Fix powered by readme link
 new c6bf261  Merge pull request #3077 from brucesw/readme_powered_by_link
65ab12c is described below

commit 65ab12c3e7605904160ba62816067bb23a0f3944
Author: Bruce Szudera Wienand 
AuthorDate: Fri Jul 5 15:01:26 2019 -0700

MINOR: Fix powered by readme link
---
 README.markdown | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/README.markdown b/README.markdown
index 9f9522d..1b0c6e2 100644
--- a/README.markdown
+++ b/README.markdown
@@ -2,7 +2,7 @@ Master Branch:
 [![Travis 
CI](https://travis-ci.org/apache/storm.svg?branch=master)](https://travis-ci.org/apache/storm)
 [![Maven 
Version](https://maven-badges.herokuapp.com/maven-central/org.apache.storm/storm-core/badge.svg)](http://search.maven.org/#search|gav|1|g:"org.apache.storm"%20AND%20a:"storm-core")
  
-Storm is a distributed realtime computation system. Similar to how Hadoop 
provides a set of general primitives for doing batch processing, Storm provides 
a set of general primitives for doing realtime computation. Storm is simple, 
can be used with any programming language, [is used by many 
companies](http://storm.apache.org/documentation/Powered-By.html), and is a lot 
of fun to use!
+Storm is a distributed realtime computation system. Similar to how Hadoop 
provides a set of general primitives for doing batch processing, Storm provides 
a set of general primitives for doing realtime computation. Storm is simple, 
can be used with any programming language, [is used by many 
companies](http://storm.apache.org/Powered-By.html), and is a lot of fun to use!
 
 The [Rationale page](http://storm.apache.org/documentation/Rationale.html) 
explains what Storm is and why it was built. [This 
presentation](http://vimeo.com/40972420) is also a good introduction to the 
project.
 



[storm] branch master updated: STORM-3465: examples-redis: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 5438a01  STORM-3465: examples-redis: fix all checkstyle warnings
 new 65d545e  Merge pull request #3083 from 
krichter722/checkstyle-redis-examples
5438a01 is described below

commit 5438a0157eb0cfeb87ffa66633b23286fcc806fe
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 18:39:42 2019 +0200

STORM-3465: examples-redis: fix all checkstyle warnings
---
 examples/storm-redis-examples/pom.xml |  2 +-
 .../storm/redis/tools/Base64ToBinaryStateMigrationUtil.java   |  1 -
 .../java/org/apache/storm/redis/topology/LookupWordCount.java | 11 ++-
 .../org/apache/storm/redis/topology/PersistentWordCount.java  |  4 ++--
 .../org/apache/storm/redis/topology/WhitelistWordCount.java   |  8 
 .../java/org/apache/storm/redis/topology/WordCounter.java |  8 +---
 .../main/java/org/apache/storm/redis/topology/WordSpout.java  |  9 +
 .../java/org/apache/storm/redis/trident/PrintFunction.java| 10 ++
 .../org/apache/storm/redis/trident/WordCountLookupMapper.java | 11 ++-
 .../org/apache/storm/redis/trident/WordCountStoreMapper.java  |  3 ++-
 .../org/apache/storm/redis/trident/WordCountTridentRedis.java |  4 +++-
 .../storm/redis/trident/WordCountTridentRedisCluster.java |  8 +---
 .../storm/redis/trident/WordCountTridentRedisClusterMap.java  |  8 +---
 .../apache/storm/redis/trident/WordCountTridentRedisMap.java  |  4 +++-
 14 files changed, 53 insertions(+), 38 deletions(-)

diff --git a/examples/storm-redis-examples/pom.xml 
b/examples/storm-redis-examples/pom.xml
index 1a8914f..4ec2347 100644
--- a/examples/storm-redis-examples/pom.xml
+++ b/examples/storm-redis-examples/pom.xml
@@ -91,7 +91,7 @@
 maven-checkstyle-plugin
 
 
-54
+0
 
 
 
diff --git 
a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/tools/Base64ToBinaryStateMigrationUtil.java
 
b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/tools/Base64ToBinaryStateMigrationUtil.java
index 68bf9dd..9f060b8 100644
--- 
a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/tools/Base64ToBinaryStateMigrationUtil.java
+++ 
b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/tools/Base64ToBinaryStateMigrationUtil.java
@@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory;
 
 import redis.clients.util.SafeEncoder;
 
-
 public class Base64ToBinaryStateMigrationUtil {
 private static final Logger LOG = 
LoggerFactory.getLogger(Base64ToBinaryStateMigrationUtil.class);
 private static final String OPTION_REDIS_HOST_SHORT = "h";
diff --git 
a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
 
b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
index 475bbab..54456ec 100644
--- 
a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
+++ 
b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
@@ -15,8 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.redis.topology;
 
+import com.google.common.collect.Lists;
+
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -36,11 +39,10 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.ITuple;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
-
 public class LookupWordCount {
 private static final String WORD_SPOUT = "WORD_SPOUT";
 private static final String LOOKUP_BOLT = "LOOKUP_BOLT";
@@ -65,7 +67,7 @@ public class LookupWordCount {
 String countStr = input.getStringByField("count");
 
 // print lookup result with low probability
-if(RANDOM.nextInt(1000) > 995) {
+if (RANDOM.nextInt(1000) > 995) {
 int count = 0;
 if (countStr != null) {
 count = Integer.parseInt(countStr);
@@ -82,8 +84,6 @@ public class LookupWordCount {
 }
 
 public static void main(String[] args) throws Exception {
-Config config = new Config();
-
 String host = TEST_REDIS_HOST;
 int port = TEST_REDIS_PORT;
 
@@ -114,6 +114,7 @@ public class LookupWordCount {
 System.out.println("Usage: LookupWordCount   (topology name)");
 return;
 }
+Config confi

[storm] branch master updated: STORM-3464: examples-mongodb: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new fe41f8d  STORM-3464: examples-mongodb: fix all checkstyle warnings
 new bc42472  Merge pull request #3082 from 
krichter722/checkstyle-mongodb-examples
fe41f8d is described below

commit fe41f8d02f80971a06e89d1095397a201888dbfb
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 18:30:38 2019 +0200

STORM-3464: examples-mongodb: fix all checkstyle warnings
---
 examples/storm-mongodb-examples/pom.xml  |  2 +-
 .../apache/storm/mongodb/topology/InsertWordCount.java   |  4 ++--
 .../apache/storm/mongodb/topology/LookupWordCount.java   |  6 +++---
 .../apache/storm/mongodb/topology/TotalWordCounter.java  | 16 +---
 .../apache/storm/mongodb/topology/UpdateWordCount.java   |  4 ++--
 .../org/apache/storm/mongodb/topology/WordCounter.java   |  8 +---
 .../org/apache/storm/mongodb/topology/WordSpout.java |  9 +
 .../org/apache/storm/mongodb/trident/PrintFunction.java  |  7 ---
 .../apache/storm/mongodb/trident/WordCountTrident.java   |  3 ++-
 .../storm/mongodb/trident/WordCountTridentMap.java   |  8 
 10 files changed, 37 insertions(+), 30 deletions(-)

diff --git a/examples/storm-mongodb-examples/pom.xml 
b/examples/storm-mongodb-examples/pom.xml
index faecbed..3766f7e 100644
--- a/examples/storm-mongodb-examples/pom.xml
+++ b/examples/storm-mongodb-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-38
+0
 
 
 
diff --git 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
index 3b27fd1..f877fca 100644
--- 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
+++ 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.mongodb.topology;
 
 import org.apache.storm.Config;
@@ -35,8 +36,6 @@ public class InsertWordCount {
 
 
 public static void main(String[] args) throws Exception {
-Config config = new Config();
-
 String url = TEST_MONGODB_URL;
 String collectionName = TEST_MONGODB_COLLECTION_NAME;
 
@@ -67,6 +66,7 @@ public class InsertWordCount {
 System.out.println("Usage: InsertWordCount   [topology name]");
 return;
 }
+Config config = new Config();
 StormSubmitter.submitTopology(topoName, config, 
builder.createTopology());
 }
 }
diff --git 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
index c9d43bd..6aaec42 100644
--- 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
+++ 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.mongodb.topology;
 
 import org.apache.storm.Config;
@@ -36,8 +37,6 @@ public class LookupWordCount {
 private static final String TEST_MONGODB_COLLECTION_NAME = "wordcount";
 
 public static void main(String[] args) throws Exception {
-Config config = new Config();
-
 String url = TEST_MONGODB_URL;
 String collectionName = TEST_MONGODB_COLLECTION_NAME;
 
@@ -70,7 +69,8 @@ public class LookupWordCount {
 System.out.println("Usage: LookupWordCount   [topology name]");
 return;
 }
-
+
+Config config = new Config();
 StormSubmitter.submitTopology(topoName, config, 
builder.createTopology());
 }
 }
diff --git 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/TotalWordCounter.java
 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/TotalWordCounter.java
index b5d20cb..148f2f9 100644
--- 
a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/TotalWordCounter.java
+++ 
b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/TotalWordCounter.java
@@ -15,8 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.sto

[storm] branch master updated: STORM-3463: examples-pmml: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 1672d45  STORM-3463: examples-pmml: fix all checkstyle warnings
 new 237b0da  Merge pull request #3081 from 
krichter722/checkstyle-pmml-examples
1672d45 is described below

commit 1672d4553b928a02b4d397c19b16ca672c07c723
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 18:23:15 2019 +0200

STORM-3463: examples-pmml: fix all checkstyle warnings
---
 examples/storm-pmml-examples/pom.xml   |  2 +-
 .../org/apache/storm/pmml/JpmmlRunnerTestTopology.java | 10 +-
 .../org/apache/storm/pmml/RawInputFromCSVSpout.java| 18 ++
 3 files changed, 16 insertions(+), 14 deletions(-)

diff --git a/examples/storm-pmml-examples/pom.xml 
b/examples/storm-pmml-examples/pom.xml
index 174fef4..b687ac1 100644
--- a/examples/storm-pmml-examples/pom.xml
+++ b/examples/storm-pmml-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-17
+0
 
 
 
diff --git 
a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
 
b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
index c330f98..95c8160 100644
--- 
a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
+++ 
b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
@@ -18,6 +18,8 @@
 
 package org.apache.storm.pmml;
 
+import com.google.common.collect.Lists;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -40,13 +42,11 @@ import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
 
-import com.google.common.collect.Lists;
-
 /**
  * Topology that loads a PMML Model and raw input data from a CSV file. The 
{@link RawInputFromCSVSpout}
  * creates a stream of tuples with raw inputs, and the {@link 
PMMLPredictorBolt} computes the predicted scores.
  *
- * The location of the PMML Model and CSV files can be specified as CLI 
argument. Alternatively, the PMML Model can also
+ * The location of the PMML Model and CSV files can be specified as CLI 
argument. Alternatively, the PMML Model can also
  * be uploaded to the Blobstore and used in the topology specifying the 
blobKey. If no arguments are given,
  * it loads the default example as described in the README file
  */
@@ -79,8 +79,8 @@ public class JpmmlRunnerTestTopology {
 private void parseArgs(String[] args) {
 if (Arrays.stream(args).anyMatch(option -> option.equals("-h"))) {
 printUsage();
-} else if (Arrays.stream(args).anyMatch(option -> option.equals("-f")) 
&&
-Arrays.stream(args).anyMatch(option -> option.equals("-b"))) {
+} else if (Arrays.stream(args).anyMatch(option -> option.equals("-f"))
+&& Arrays.stream(args).anyMatch(option -> 
option.equals("-b"))) {
 System.out.println("Please specify only one option of [-b, -f]");
 printUsage();
 } else {
diff --git 
a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/RawInputFromCSVSpout.java
 
b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/RawInputFromCSVSpout.java
index 444837b..b5e7636 100644
--- 
a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/RawInputFromCSVSpout.java
+++ 
b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/RawInputFromCSVSpout.java
@@ -18,14 +18,6 @@
 
 package org.apache.storm.pmml;
 
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.apache.storm.tuple.Fields;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -37,6 +29,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class RawInputFromCSVSpout extends BaseRichSpout {
 private static final Logger LOG = 
LoggerFactory.getLogger(RawInputFromCSVSpout.class);
 



[storm] branch master updated: STORM-3462: examples-hive: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 5c610a1  STORM-3462: examples-hive: fix all checkstyle warnings
 new efb4047  Merge pull request #3080 from 
krichter722/checkstyle-hive-examples
5c610a1 is described below

commit 5c610a1f110245a70e29c0e4c85d253c37b7e2be
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 18:18:47 2019 +0200

STORM-3462: examples-hive: fix all checkstyle warnings
---
 examples/storm-hive-examples/pom.xml   |  2 +-
 .../storm/hive/bolt/BucketTestHiveTopology.java| 50 --
 .../org/apache/storm/hive/bolt/HiveTopology.java   | 19 
 .../storm/hive/bolt/HiveTopologyPartitioned.java   | 19 
 .../storm/hive/trident/TridentHiveTopology.java| 27 ++--
 5 files changed, 61 insertions(+), 56 deletions(-)

diff --git a/examples/storm-hive-examples/pom.xml 
b/examples/storm-hive-examples/pom.xml
index 7fe757d..10e2112 100644
--- a/examples/storm-hive-examples/pom.xml
+++ b/examples/storm-hive-examples/pom.xml
@@ -97,7 +97,7 @@
 maven-checkstyle-plugin
 
 
-67
+0
 
 
 
diff --git 
a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
 
b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
index 85bbde7..61ea97c 100644
--- 
a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
+++ 
b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
@@ -48,33 +48,33 @@ public class BucketTestHiveTopology {
 public static void main(String[] args) throws Exception {
 if ((args == null) || (args.length < 7)) {
 System.out.println("Usage: BucketTestHiveTopology metastoreURI "
-+ "dbName tableName dataFileLocation hiveBatchSize " +
-"hiveTickTupl]eIntervalSecs workers  [topologyNamey] 
[keytab file]"
++ "dbName tableName dataFileLocation hiveBatchSize "
++ "hiveTickTupl]eIntervalSecs workers  [topologyNamey] 
[keytab file]"
 + " [principal name] ");
 System.exit(1);
 }
-String metaStoreURI = args[0];
+String metaStoreUri = args[0];
 String dbName = args[1];
 String tblName = args[2];
-String sourceFileLocation = args[3];
 Integer hiveBatchSize = Integer.parseInt(args[4]);
 Integer hiveTickTupleIntervalSecs = Integer.parseInt(args[5]);
 Integer workers = Integer.parseInt(args[6]);
-String[] colNames = { "ss_sold_date_sk", "ss_sold_time_sk", 
"ss_item_sk",
-"ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk",
-"ss_store_sk", "ss_promo_sk", "ss_ticket_number", 
"ss_quantity",
-"ss_wholesale_cost", "ss_list_price", "ss_sales_price",
-"ss_ext_discount_amt", "ss_ext_sales_price",
-"ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax",
-"ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax",
-"ss_net_profit" };
+String[] colNames = {
+"ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk",
+"ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk",
+"ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity",
+"ss_wholesale_cost", "ss_list_price", "ss_sales_price",
+"ss_ext_discount_amt", "ss_ext_sales_price",
+"ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax",
+"ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax",
+"ss_net_profit"
+};
 Config config = new Config();
 config.setNumWorkers(workers);
-UserDataSpout spout = new 
UserDataSpout().withDataFile(sourceFileLocation);
 DelimitedRecordHiveMapper mapper = new DelimitedRecordHiveMapper()
 .withColumnFields(new 
Fields(colNames)).withTimeAsPartitionField("/MM/dd");
 HiveOptions hiveOptions;
-hiveOptions = new HiveOptions(metaStoreURI,dbName,

[storm] branch master updated: STORM-3461: examples-hdfs: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 918e894  STORM-3461: examples-hdfs: fix all checkstyle warnings
 new 5b1f70d  Merge pull request #3079 from 
krichter722/checkstyle-hdfs-examples
918e894 is described below

commit 918e894da7ac528cf2f350babe3878c36107b0ba
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 18:09:45 2019 +0200

STORM-3461: examples-hdfs: fix all checkstyle warnings
---
 examples/storm-hdfs-examples/pom.xml   |  2 +-
 .../apache/storm/hdfs/bolt/HdfsFileTopology.java   |  1 +
 .../storm/hdfs/bolt/SequenceFileTopology.java  |  1 +
 .../apache/storm/hdfs/spout/HdfsSpoutTopology.java | 33 +++---
 .../storm/hdfs/trident/TridentFileTopology.java| 10 +++
 .../hdfs/trident/TridentSequenceTopology.java  | 10 +++
 6 files changed, 29 insertions(+), 28 deletions(-)

diff --git a/examples/storm-hdfs-examples/pom.xml 
b/examples/storm-hdfs-examples/pom.xml
index f1e11da..697a22c 100644
--- a/examples/storm-hdfs-examples/pom.xml
+++ b/examples/storm-hdfs-examples/pom.xml
@@ -97,7 +97,7 @@
 maven-checkstyle-plugin
 
 
-29
+0
 
 
 
diff --git 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
index e0e3057..01a446c 100644
--- 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
+++ 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
@@ -101,6 +101,7 @@ public class HdfsFileTopology {
 try {
 Thread.sleep(seconds * 1000);
 } catch (InterruptedException e) {
+//ignore
 }
 }
 
diff --git 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
index 2a58249..90e0aea 100644
--- 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
+++ 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
@@ -104,6 +104,7 @@ public class SequenceFileTopology {
 try {
 Thread.sleep(seconds * 1000);
 } catch (InterruptedException e) {
+//ignore
 }
 }
 
diff --git 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
index e3282d8..890b7ab 100644
--- 
a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
+++ 
b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
@@ -63,25 +63,13 @@ public class HdfsSpoutTopology {
 }
 
 // 1 - parse cmd line args
-String topologyName = args[0];
 String hdfsUri = args[1];
 String fileFormat = args[2];
 String sourceDir = args[3];
 String archiveDir = args[4];
 String badDir = args[5];
-int spoutNum = Integer.parseInt(args[6]);
-
-// 2 - create and configure spout and bolt
-ConstBolt bolt = new ConstBolt();
-
-HdfsSpout spout = new 
HdfsSpout().withOutputFields(TextFileReader.defaultFields)
- .setReaderType(fileFormat)
- .setHdfsUri(hdfsUri)
- .setSourceDir(sourceDir)
- .setArchiveDir(archiveDir)
- .setBadFilesDir(badDir);
 
-// 3 - Create and configure topology
+// 2 - Create and configure topology
 Config conf = new Config();
 conf.setNumWorkers(1);
 conf.setNumAckers(1);
@@ -90,15 +78,24 @@ public class HdfsSpoutTopology {
 conf.registerMetricsConsumer(LoggingMetricsConsumer.class);
 
 TopologyBuilder builder = new TopologyBuilder();
+HdfsSpout spout = new 
HdfsSpout().withOutputFields(TextFileReader.defaultFields)
+.setReaderType(fileFormat)
+.setHdfsUri(hdfsUri)
+.setSourceDir(sourceDir)
+.setArchiveDir(archiveDir)
+.setBadFilesDir(badDir);
+int spoutNum = Integer.parseInt(args[6]);
 builder.setSpout(SPOUT_ID, spout, spoutNum);
+ConstBolt bolt = new ConstBolt();
 builder.setBolt(BOLT_ID, bolt, 1).shuffleGrouping(SPOUT_ID);
 
-// 4 - submit topology, wait

[storm] branch master updated: STORM-3460: examples-opentsdb: fix all checkstyle warnings

2019-07-07 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new f446246  STORM-3460: examples-opentsdb: fix all checkstyle warnings
 new f5ddeb2  Merge pull request #3078 from 
krichter722/checkstyle-opentsdb-examples
f446246 is described below

commit f44624685ca5c12963c2939ca58eb0db10fbeb24
Author: Karl-Philipp Richter 
AuthorDate: Sat Jul 6 17:59:31 2019 +0200

STORM-3460: examples-opentsdb: fix all checkstyle warnings
---
 examples/storm-opentsdb-examples/pom.xml   |  2 +-
 .../org/apache/storm/opentsdb/MetricGenBatchSpout.java | 14 --
 .../java/org/apache/storm/opentsdb/MetricGenSpout.java | 10 ++
 .../apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java  |  3 ++-
 .../storm/opentsdb/SampleOpenTsdbTridentTopology.java  |  3 ++-
 5 files changed, 19 insertions(+), 13 deletions(-)

diff --git a/examples/storm-opentsdb-examples/pom.xml 
b/examples/storm-opentsdb-examples/pom.xml
index fba4737..90e9d76 100644
--- a/examples/storm-opentsdb-examples/pom.xml
+++ b/examples/storm-opentsdb-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-18
+0
 
 
 
diff --git 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenBatchSpout.java
 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenBatchSpout.java
index 4e57d9a..85b2fb2 100644
--- 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenBatchSpout.java
+++ 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenBatchSpout.java
@@ -16,14 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.storm.opentsdb;
 
 import com.google.common.collect.Lists;
-import org.apache.storm.Config;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.spout.IBatchSpout;
-import org.apache.storm.tuple.Fields;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -32,6 +28,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.storm.Config;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.spout.IBatchSpout;
+import org.apache.storm.tuple.Fields;
+
 /**
  * BatchSpout implementation for metrics generation.
  */
@@ -52,7 +54,7 @@ public class MetricGenBatchSpout implements IBatchSpout {
 @Override
 public void emitBatch(long batchId, TridentCollector collector) {
 List> values;
-if(batches.containsKey(batchId)) {
+if (batches.containsKey(batchId)) {
 values = batches.get(batchId);
 } else {
 values = new ArrayList<>();
diff --git 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenSpout.java
 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenSpout.java
index 09904ee..b963261 100644
--- 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenSpout.java
+++ 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/MetricGenSpout.java
@@ -16,9 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.storm.opentsdb;
 
 import com.google.common.collect.Lists;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Random;
+
 import org.apache.storm.opentsdb.bolt.TupleOpenTsdbDatapointMapper;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -26,10 +32,6 @@ import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 
-import java.util.Collections;
-import java.util.Map;
-import java.util.Random;
-
 /**
  * Spout to generate tuples containing metric data.
  */
diff --git 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
index 009366f..23a086c 100644
--- 
a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
+++ 
b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.storm.opentsdb;
 
 import java.util.Co

[storm] branch master updated: STORM-3457: hdfs: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 01b299b  STORM-3457: hdfs: fix all checkstyle warnings
 new 98ac65f  Merge pull request #3075 from krichter722/checkstyle-hdfs
01b299b is described below

commit 01b299bbd4c0a749f122befc7ec5d275832fdfe6
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 22:51:35 2019 +0200

STORM-3457: hdfs: fix all checkstyle warnings
---
 external/storm-hdfs/pom.xml|  2 +-
 .../storm/hdfs/avro/AbstractAvroSerializer.java|  2 +-
 .../storm/hdfs/avro/ConfluentAvroSerializer.java   |  2 +-
 .../storm/hdfs/avro/FixedAvroSerializer.java   |  2 +-
 .../apache/storm/hdfs/bolt/AbstractHdfsBolt.java   | 22 ---
 .../apache/storm/hdfs/bolt/SequenceFileBolt.java   |  4 +-
 .../hdfs/bolt/format/DefaultFileNameFormat.java|  9 +--
 .../hdfs/bolt/format/DelimitedRecordFormat.java| 11 +---
 .../storm/hdfs/bolt/format/FileNameFormat.java |  1 -
 .../storm/hdfs/bolt/format/RecordFormat.java   |  5 +-
 .../storm/hdfs/bolt/format/SequenceFormat.java | 14 +
 .../hdfs/bolt/format/SimpleFileNameFormat.java |  4 +-
 .../hdfs/bolt/rotation/FileRotationPolicy.java |  8 +--
 .../hdfs/bolt/rotation/FileSizeRotationPolicy.java |  4 +-
 .../storm/hdfs/bolt/sync/CountSyncPolicy.java  |  1 -
 .../storm/hdfs/common/AbstractHDFSWriter.java  | 19 +++---
 .../hdfs/common/AvroGenericRecordHDFSWriter.java   |  2 +-
 .../org/apache/storm/hdfs/common/HDFSWriter.java   |  1 +
 .../org/apache/storm/hdfs/common/Partitioner.java  |  3 +-
 .../storm/hdfs/common/rotation/RotationAction.java |  1 -
 .../java/org/apache/storm/hdfs/spout/Configs.java  | 39 
 .../java/org/apache/storm/hdfs/spout/DirLock.java  | 15 +++--
 .../java/org/apache/storm/hdfs/spout/FileLock.java | 69 +-
 .../org/apache/storm/hdfs/spout/FileOffset.java|  4 +-
 .../org/apache/storm/hdfs/spout/FileReader.java|  3 +-
 .../org/apache/storm/hdfs/spout/HdfsSpout.java | 35 +--
 .../storm/hdfs/spout/SequenceFileReader.java   | 38 ++--
 .../apache/storm/hdfs/spout/TextFileReader.java| 20 +++
 .../org/apache/storm/hdfs/trident/HdfsState.java   | 41 ++---
 .../hdfs/trident/format/DefaultFileNameFormat.java |  8 +--
 .../hdfs/trident/format/DelimitedRecordFormat.java | 11 
 .../storm/hdfs/trident/format/FileNameFormat.java  |  2 -
 .../storm/hdfs/trident/format/RecordFormat.java|  5 +-
 .../storm/hdfs/trident/format/SequenceFormat.java  | 14 +
 .../hdfs/trident/format/SimpleFileNameFormat.java  |  4 +-
 .../hdfs/trident/rotation/FileRotationPolicy.java  |  4 +-
 .../trident/rotation/FileSizeRotationPolicy.java   |  5 +-
 .../hdfs/trident/rotation/TimedRotationPolicy.java |  2 +-
 .../storm/hdfs/trident/sync/CountSyncPolicy.java   |  1 -
 .../org/apache/storm/hdfs/spout/TestFileLock.java  |  2 +-
 40 files changed, 187 insertions(+), 252 deletions(-)

diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 60c74a9..d0db240 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -275,7 +275,7 @@
 maven-checkstyle-plugin
 
 
-189
+0
 
 
 
diff --git 
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java
 
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java
index adb842a..da7ab74 100644
--- 
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java
+++ 
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/AbstractAvroSerializer.java
@@ -57,7 +57,7 @@ public abstract class AbstractAvroSerializer extends 
Serializer aClass) {
+public GenericContainer read(Kryo kryo, Input input, 
Class someClass) {
 Schema theSchema = this.getSchema(input.readString());
 GenericDatumReader reader = new 
GenericDatumReader<>(theSchema);
 Decoder decoder = DecoderFactory
diff --git 
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
 
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
index 17f3eb7..128d4ff 100644
--- 
a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
+++ 
b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
@@ -27,7 +27,7 @@ import org.apache.avro.Schema;
  */
 public class ConfluentAvroSerializer extends AbstractAvroSerializer {
 
-final private String url;
+private final String url;
 private SchemaRegistryClient theClient;
 
 /**
diff --git 
a/external/storm-hdfs/src/main/java/org/apache/storm/hdf

[storm] branch master updated: STORM-3451: jdbc: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 32ccd43  STORM-3451: jdbc: fix all checkstyle warnings
 new 224f75a  Merge pull request #3069 from krichter722/checkstyle-jdbc
32ccd43 is described below

commit 32ccd43e37e5fab42b8dc8d07b44180f6bda0a02
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 20:48:41 2019 +0200

STORM-3451: jdbc: fix all checkstyle warnings
---
 external/storm-jdbc/pom.xml|  2 +-
 .../java/org/apache/storm/jdbc/common/Column.java  | 35 --
 .../storm/jdbc/common/ConnectionProvider.java  |  2 +-
 .../jdbc/common/HikariCPConnectionProvider.java|  5 ++--
 .../apache/storm/jdbc/mapper/JdbcLookupMapper.java |  1 -
 .../org/apache/storm/jdbc/mapper/JdbcMapper.java   |  3 +-
 .../storm/jdbc/mapper/SimpleJdbcLookupMapper.java  |  1 -
 .../apache/storm/jdbc/trident/state/JdbcState.java | 13 
 .../storm/jdbc/trident/state/JdbcStateFactory.java |  2 +-
 9 files changed, 34 insertions(+), 30 deletions(-)

diff --git a/external/storm-jdbc/pom.xml b/external/storm-jdbc/pom.xml
index f868739..2b31123 100644
--- a/external/storm-jdbc/pom.xml
+++ b/external/storm-jdbc/pom.xml
@@ -90,7 +90,7 @@
 maven-checkstyle-plugin
 
 
-36
+0
 
 
 
diff --git 
a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/Column.java 
b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/Column.java
index c796d3b..0b3498e 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/Column.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/Column.java
@@ -12,7 +12,6 @@
 
 package org.apache.storm.jdbc.common;
 
-
 import java.io.Serializable;
 
 /**
@@ -27,18 +26,16 @@ import java.io.Serializable;
  *  
  * 
  *
- * The following class can be used to represent the data in the table as
- * 
+ * The following class can be used to represent the data in the table as
+ * 
  * 
- *
- * @param 
  */
 public class Column implements Serializable {
 
@@ -76,13 +73,21 @@ public class Column implements Serializable {
 
 @Override
 public boolean equals(Object o) {
-if (this == o) return true;
-if (!(o instanceof Column)) return false;
+if (this == o) {
+return true;
+}
+if (!(o instanceof Column)) {
+return false;
+}
 
 Column column = (Column) o;
 
-if (sqlType != column.sqlType) return false;
-if (!columnName.equals(column.columnName)) return false;
+if (sqlType != column.sqlType) {
+return false;
+}
+if (!columnName.equals(column.columnName)) {
+return false;
+}
 return val != null ? val.equals(column.val) : column.val == null;
 
 }
@@ -97,10 +102,10 @@ public class Column implements Serializable {
 
 @Override
 public String toString() {
-return "Column{" +
-   "columnName='" + columnName + '\'' +
-   ", val=" + val +
-   ", sqlType=" + sqlType +
-   '}';
+return "Column{"
++ "columnName='" + columnName + '\''
++ ", val=" + val
++ ", sqlType=" + sqlType
++ '}';
 }
 }
diff --git 
a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/ConnectionProvider.java
 
b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/ConnectionProvider.java
index 6804b5d..39b3ddf 100644
--- 
a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/ConnectionProvider.java
+++ 
b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/ConnectionProvider.java
@@ -25,7 +25,7 @@ public interface ConnectionProvider extends Serializable {
 void prepare();
 
 /**
- *
+ * Get connection.
  * @return a DB connection over which the queries can be executed.
  */
 Connection getConnection();
diff --git 
a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/HikariCPConnectionProvider.java
 
b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/HikariCPConnectionProvider.java
index dbb0009..71c9488 100644
--- 
a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/HikariCPConnectionProvider.java
+++ 
b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/common/HikariCPConnectionProvider.java
@@ -21,14 +21,15 @@ import java.util.Properties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class Hikar

[storm] branch master updated: STORM-3456: cassandra: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4f53227  STORM-3456: cassandra: fix all checkstyle warnings
 new 8cd3a53  Merge pull request #3074 from krichter722/checkstyle-cassandra
4f53227 is described below

commit 4f532273f9d63ac560561d13be31d5242d4d82b4
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 22:13:23 2019 +0200

STORM-3456: cassandra: fix all checkstyle warnings
---
 external/storm-cassandra/pom.xml   |  2 +-
 .../cassandra/AbstractExecutionResultHandler.java  |  4 ++-
 .../cassandra/BaseExecutionResultHandler.java  |  3 ++-
 .../apache/storm/cassandra/CassandraContext.java   |  3 ---
 .../storm/cassandra/DynamicStatementBuilder.java   |  6 +++--
 .../storm/cassandra/Murmur3StreamGrouping.java |  8 +++---
 .../storm/cassandra/bolt/BaseCassandraBolt.java| 27 +--
 .../cassandra/bolt/BatchCassandraWriterBolt.java   | 15 +--
 .../storm/cassandra/bolt/CassandraWriterBolt.java  |  2 --
 .../storm/cassandra/bolt/GroupingBatchBuilder.java |  4 ---
 .../storm/cassandra/bolt/PairStatementTuple.java   |  2 --
 .../storm/cassandra/client/CassandraConf.java  |  2 +-
 .../storm/cassandra/client/impl/DefaultClient.java |  5 ++--
 .../storm/cassandra/context/BaseBeanFactory.java   |  4 ++-
 .../storm/cassandra/context/BeanFactory.java   |  2 --
 .../apache/storm/cassandra/context/WorkerCtx.java  | 12 +
 .../storm/cassandra/executor/AsyncExecutor.java| 21 ---
 .../executor/ExecutionResultCollector.java |  8 --
 .../executor/impl/BatchAsyncResultHandler.java |  6 ++---
 .../executor/impl/SingleAsyncResultHandler.java|  6 ++---
 .../query/AyncCQLResultSetValuesMapper.java|  3 ++-
 .../query/BaseCQLStatementTupleMapper.java |  2 +-
 .../cassandra/query/CQLResultSetValuesMapper.java  |  4 +--
 .../storm/cassandra/query/CQLStatementBuilder.java |  2 +-
 .../cassandra/query/CQLStatementTupleMapper.java   |  2 ++
 .../org/apache/storm/cassandra/query/Column.java   | 26 +++---
 .../apache/storm/cassandra/query/ContextQuery.java | 10 ---
 .../apache/storm/cassandra/query/CqlMapper.java|  6 +++--
 .../builder/BoundCQLStatementMapperBuilder.java|  7 +++--
 .../builder/SimpleCQLStatementMapperBuilder.java   | 11 
 .../query/impl/BatchCQLStatementTupleMapper.java   |  7 +++--
 .../query/impl/BoundCQLStatementTupleMapper.java   |  7 +
 .../query/impl/ObjectMapperCqlStatementMapper.java |  5 ++--
 .../query/impl/PreparedStatementBinder.java|  4 +--
 .../cassandra/query/impl/RoutingKeyGenerator.java  |  1 -
 .../query/impl/SimpleCQLStatementMapper.java   |  4 +--
 .../cassandra/query/selector/FieldSelector.java| 12 -
 .../trident/state/CassandraBackingMap.java | 13 -
 .../trident/state/CassandraMapStateFactory.java|  3 +--
 .../cassandra/trident/state/CassandraQuery.java|  3 ---
 .../cassandra/trident/state/CassandraState.java|  5 ++--
 .../trident/state/CassandraStateFactory.java   |  3 ---
 .../trident/state/CassandraStateUpdater.java   |  3 ---
 .../trident/state/MapStateFactoryBuilder.java  | 31 +++---
 .../trident/state/OpaqueTupleStateMapper.java  |  1 +
 .../state/TransactionalTupleStateMapper.java   |  1 +
 .../state/TridentAyncCQLResultSetValuesMapper.java |  1 +
 .../state/TridentResultSetValuesMapper.java|  3 ---
 48 files changed, 154 insertions(+), 168 deletions(-)

diff --git a/external/storm-cassandra/pom.xml b/external/storm-cassandra/pom.xml
index f670a6e..495f9d5 100644
--- a/external/storm-cassandra/pom.xml
+++ b/external/storm-cassandra/pom.xml
@@ -132,7 +132,7 @@
 maven-checkstyle-plugin
 
 
-159
+0
 
 
 
diff --git 
a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/AbstractExecutionResultHandler.java
 
b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/AbstractExecutionResultHandler.java
index 3b32749..bb397b7 100644
--- 
a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/AbstractExecutionResultHandler.java
+++ 
b/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/AbstractExecutionResultHandler.java
@@ -48,6 +48,8 @@ public abstract class AbstractExecutionResultHandler 
implements ExecutionResultH
 
 @Override
 public void onThrowable(Throwable t, OutputCollector collector, 
List tl) {
-for (Tuple i : tl) onThrowable(t, collector, i);
+for (Tuple i : tl) {
+onThrowable(t, collector, i);
+}
 }
 }
diff --git 
a/external/storm-cassandra/src/main/java/org/apache/storm/cassandra/BaseExecutionResultHandler.java

[storm] branch master updated: STORM-3458: starter: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 2ac9e2e  STORM-3458: starter: fix all checkstyle warnings
 new addc1df  Merge pull request #3076 from krichter722/checkstyle-starter
2ac9e2e is described below

commit 2ac9e2e62a6fbb82bcff94c7015241f8135b6a4b
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 23:28:13 2019 +0200

STORM-3458: starter: fix all checkstyle warnings
---
 examples/storm-starter/pom.xml |  2 +-
 .../apache/storm/starter/BasicDRPCTopology.java|  1 +
 .../starter/BlobStoreAPIWordCountTopology.java | 13 ++--
 .../apache/storm/starter/ExclamationTopology.java  |  8 +--
 .../storm/starter/FastWordCountTopology.java   | 21 +++---
 .../apache/storm/starter/InOrderDeliveryTest.java  | 27 
 .../org/apache/storm/starter/LambdaTopology.java   | 13 
 .../jvm/org/apache/storm/starter/ManualDRPC.java   |  2 +
 .../storm/starter/MultipleLoggerTopology.java  | 38 +--
 .../storm/starter/PersistentWindowingTopology.java |  4 +-
 .../src/jvm/org/apache/storm/starter/Prefix.java   | 28 
 .../org/apache/storm/starter/ReachTopology.java| 74 --
 .../starter/ResourceAwareExampleTopology.java  |  8 +--
 .../org/apache/storm/starter/RollingTopWords.java  | 11 ++--
 .../apache/storm/starter/SingleJoinExample.java|  6 +-
 .../storm/starter/SkewedRollingTopWords.java   | 16 ++---
 .../apache/storm/starter/bolt/SingleJoinBolt.java  | 60 +-
 .../storm/starter/bolt/SlidingWindowSumBolt.java   |  2 +-
 .../storm/starter/spout/RandomSentenceSpout.java   | 12 ++--
 .../storm/starter/streams/AggregateExample.java|  2 +-
 .../storm/starter/streams/BranchExample.java   | 22 +++
 .../streams/GroupByKeyAndWindowExample.java|  2 +-
 .../apache/storm/starter/streams/JoinExample.java  |  6 +-
 .../storm/starter/streams/StateQueryExample.java   |  8 +--
 .../storm/starter/streams/StatefulWordCount.java   |  8 +--
 .../storm/starter/streams/TypedTupleExample.java   |  9 +--
 .../storm/starter/streams/WindowedWordCount.java   |  2 +-
 .../storm/starter/streams/WordCountToBolt.java |  2 +-
 .../starter/tools/RankableObjectWithFields.java|  3 +-
 .../org/apache/storm/starter/tools/Rankings.java   |  3 +-
 .../storm/starter/tools/SlidingWindowCounter.java  |  1 +
 .../storm/starter/tools/SlotBasedCounter.java  |  2 -
 .../storm/starter/trident/DebugMemoryMapState.java |  6 +-
 .../trident/TridentMinMaxOfDevicesTopology.java| 60 +-
 .../trident/TridentMinMaxOfVehiclesTopology.java   | 39 ++--
 .../apache/storm/starter/trident/TridentReach.java | 46 --
 36 files changed, 300 insertions(+), 267 deletions(-)

diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 97a6610..3835bbc 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -229,7 +229,7 @@
   maven-checkstyle-plugin
   
   
-  263
+  0
   
   
 
diff --git 
a/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
 
b/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
index 6bb787b..70e03a4 100644
--- 
a/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
+++ 
b/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
@@ -29,6 +29,7 @@ import org.apache.storm.utils.DRPCClient;
  *
  * @see http://storm.apache.org/documentation/Distributed-RPC.html;>Distributed 
RPC
  */
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class BasicDRPCTopology {
 public static void main(String[] args) throws Exception {
 Config conf = new Config();
diff --git 
a/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
 
b/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
index caa751a..46df78e 100644
--- 
a/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
+++ 
b/examples/storm-starter/src/jvm/org/apache/storm/starter/BlobStoreAPIWordCountTopology.java
@@ -54,6 +54,7 @@ import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class BlobStoreAPIWordCountTopology {
 private static final Logger LOG = 
LoggerFactory.getLogger(BlobStoreAPIWordCountTopology.class);
 private static ClientBlobStore store; // Client API to invoke blob store 
API functionality
@@ -70,10 +71,10 @@ public class BlobStoreAPIWordCountTopology {
 // storm blobstore create --file blacklist.txt --acl o::rwa key
 

[storm] branch master updated: STORM-3455: solr: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new c1431fb  STORM-3455: solr: fix all checkstyle warnings
 new 351d219  Merge pull request #3073 from krichter722/checkstyle-solr
c1431fb is described below

commit c1431fb0f34de2b16bcb03fcc74f592d6099c571
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 21:38:01 2019 +0200

STORM-3455: solr: fix all checkstyle warnings
---
 external/storm-solr/pom.xml|  2 +-
 .../org/apache/storm/solr/bolt/SolrUpdateBolt.java | 12 ++--
 .../org/apache/storm/solr/config/CountBasedCommit.java |  2 +-
 .../java/org/apache/storm/solr/config/SolrConfig.java  |  3 +++
 .../org/apache/storm/solr/mapper/SolrFieldsMapper.java | 14 +++---
 .../org/apache/storm/solr/mapper/SolrJsonMapper.java   |  4 +---
 .../java/org/apache/storm/solr/schema/CopyField.java   |  8 
 .../main/java/org/apache/storm/solr/schema/Field.java  |  8 
 .../java/org/apache/storm/solr/schema/FieldType.java   | 10 +-
 .../main/java/org/apache/storm/solr/schema/Schema.java | 18 +-
 .../apache/storm/solr/schema/SolrFieldTypeFinder.java  | 10 +-
 .../solr/schema/builder/RestJsonSchemaBuilder.java |  4 +++-
 .../solr/schema/builder/RestJsonSchemaBuilderV2.java   |  2 +-
 .../java/org/apache/storm/solr/trident/SolrState.java  |  4 ++--
 .../apache/storm/solr/trident/SolrStateFactory.java|  2 +-
 15 files changed, 53 insertions(+), 50 deletions(-)

diff --git a/external/storm-solr/pom.xml b/external/storm-solr/pom.xml
index 14af06e..5deee1c 100644
--- a/external/storm-solr/pom.xml
+++ b/external/storm-solr/pom.xml
@@ -124,7 +124,7 @@
 maven-checkstyle-plugin
 
 
-47
+0
 
 
 
diff --git 
a/external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java
 
b/external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java
index f6c7c60..6706634 100644
--- 
a/external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java
+++ 
b/external/storm-solr/src/main/java/org/apache/storm/solr/bolt/SolrUpdateBolt.java
@@ -60,9 +60,9 @@ public class SolrUpdateBolt extends 
BaseTickTupleAwareRichBolt {
 this.solrConfig = solrConfig;
 this.solrMapper = solrMapper;
 this.commitStgy = commitStgy;
-LOG.debug("Created {} with the following configuration: " +
-  "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = {}]",
-  this.getClass().getSimpleName(), solrConfig, solrMapper, 
commitStgy);
+LOG.debug("Created {} with the following configuration: "
++ "[SolrConfig = {}], [SolrMapper = {}], [CommitStgy = 
{}]",
+this.getClass().getSimpleName(), solrConfig, solrMapper, 
commitStgy);
 }
 
 @Override
@@ -78,9 +78,9 @@ public class SolrUpdateBolt extends 
BaseTickTupleAwareRichBolt {
 
 private int capacity() {
 final int defArrListCpcty = 10;
-return (commitStgy instanceof CountBasedCommit) ?
-((CountBasedCommit) commitStgy).getThreshold() :
-defArrListCpcty;
+return (commitStgy instanceof CountBasedCommit)
+? ((CountBasedCommit) commitStgy).getThreshold()
+: defArrListCpcty;
 }
 
 @Override
diff --git 
a/external/storm-solr/src/main/java/org/apache/storm/solr/config/CountBasedCommit.java
 
b/external/storm-solr/src/main/java/org/apache/storm/solr/config/CountBasedCommit.java
index 1a9c967..3fdcc89 100644
--- 
a/external/storm-solr/src/main/java/org/apache/storm/solr/config/CountBasedCommit.java
+++ 
b/external/storm-solr/src/main/java/org/apache/storm/solr/config/CountBasedCommit.java
@@ -21,7 +21,7 @@ public class CountBasedCommit implements SolrCommitStrategy {
 private int count;
 
 /**
- * Initializes a count based commit strategy with the specified threshold
+ * Initializes a count based commit strategy with the specified threshold.
  *
  * @param threshold  The commit threshold, defining when 
SolrInputDocuments should be committed to Solr
  * */
diff --git 
a/external/storm-solr/src/main/java/org/apache/storm/solr/config/SolrConfig.java
 
b/external/storm-solr/src/main/java/org/apache/storm/solr/config/SolrConfig.java
index bd05e9d..8b36bf2 100644
--- 
a/external/storm-solr/src/main/java/org/apache/storm/solr/config/SolrConfig.java
+++ 
b/external/storm-solr/src/main/java/org/apache/storm/solr/config/SolrConfig.java
@@ -25,6 +25,7 @@ public class SolrConfig implements Serializable {
 private final boolean enableKerberos;
 
 /**
+ * Constructor.
  * @param zkHos

[storm] branch master updated: STORM-3454: hbase-examples: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 247d7f0  STORM-3454: hbase-examples: fix all checkstyle warnings
 new f2a4d06  Merge pull request #3072 from 
krichter722/checkstyle-hbase-example
247d7f0 is described below

commit 247d7f0038de23c50c7f5b1dca851ac30e0adfab
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 21:28:07 2019 +0200

STORM-3454: hbase-examples: fix all checkstyle warnings
---
 examples/storm-hbase-examples/pom.xml  |  2 +-
 .../java/org/apache/storm/hbase/topology/LookupWordCount.java  | 10 +-
 .../org/apache/storm/hbase/topology/PersistentWordCount.java   |  7 ---
 .../java/org/apache/storm/hbase/topology/TotalWordCounter.java |  4 ++--
 .../java/org/apache/storm/hbase/topology/WordCountClient.java  |  2 +-
 .../org/apache/storm/hbase/topology/WordCountValueMapper.java  |  7 +++
 .../main/java/org/apache/storm/hbase/topology/WordCounter.java |  4 ++--
 7 files changed, 18 insertions(+), 18 deletions(-)

diff --git a/examples/storm-hbase-examples/pom.xml 
b/examples/storm-hbase-examples/pom.xml
index dd733dc..65b71fe 100644
--- a/examples/storm-hbase-examples/pom.xml
+++ b/examples/storm-hbase-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-16
+0
 
 
 
diff --git 
a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
 
b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
index 1c365c4..7f3afb1 100644
--- 
a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
+++ 
b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
@@ -38,7 +38,6 @@ public class LookupWordCount {
 config.put("hbase.conf", hbConf);
 
 WordSpout spout = new WordSpout();
-TotalWordCounter totalBolt = new TotalWordCounter();
 
 SimpleHBaseMapper mapper = new 
SimpleHBaseMapper().withRowKeyField("word");
 HBaseProjectionCriteria projectionCriteria = new 
HBaseProjectionCriteria();
@@ -46,14 +45,15 @@ public class LookupWordCount {
 
 WordCountValueMapper rowToTupleMapper = new WordCountValueMapper();
 
-HBaseLookupBolt hBaseLookupBolt = new HBaseLookupBolt("WordCount", 
mapper, rowToTupleMapper)
-.withConfigKey("hbase.conf")
-.withProjectionCriteria(projectionCriteria);
+HBaseLookupBolt lookupBolt = new HBaseLookupBolt("WordCount", mapper, 
rowToTupleMapper)
+.withConfigKey("hbase.conf")
+.withProjectionCriteria(projectionCriteria);
 
 //wordspout -> lookupbolt -> totalCountBolt
 TopologyBuilder builder = new TopologyBuilder();
 builder.setSpout(WORD_SPOUT, spout, 1);
-builder.setBolt(LOOKUP_BOLT, hBaseLookupBolt, 
1).shuffleGrouping(WORD_SPOUT);
+builder.setBolt(LOOKUP_BOLT, lookupBolt, 
1).shuffleGrouping(WORD_SPOUT);
+TotalWordCounter totalBolt = new TotalWordCounter();
 builder.setBolt(TOTAL_COUNT_BOLT, totalBolt, 
1).fieldsGrouping(LOOKUP_BOLT, new Fields("columnName"));
 String topoName = "test";
 if (args.length == 1) {
diff --git 
a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
 
b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
index 015ce85..0c929a3 100644
--- 
a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
+++ 
b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
@@ -12,7 +12,6 @@
 
 package org.apache.storm.hbase.topology;
 
-
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.storm.Config;
@@ -63,8 +62,10 @@ public class PersistentWordCount {
 if (args.length == 2) {
 topoName = args[0];
 } else if (args.length == 4) {
-System.out.println("hdfs url: " + args[0] + ", keytab file: " + 
args[2] +
-   ", principal name: " + args[3] + ", toplogy 
name: " + args[1]);
+System.out.println("hdfs url: " + args[0]
++ ", keytab file: " + args[2]
++ ", principal name: " + args[3]
++ ", toplogy name: " + args[1]);
 hbConf.put(HBaseSecurityUtil.STORM_KEYTAB_FILE_KEY, args[2]);
 hbConf.put(HBaseSecur

[storm] branch master updated: STORM-3453: eventhubs: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4764432  STORM-3453: eventhubs: fix all checkstyle warnings
 new 1bb13c6  Merge pull request #3071 from krichter722/checkstyle-eventhubs
4764432 is described below

commit 4764432f1692a3a6f5e89a1bbf5e430481adb0f4
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 21:16:21 2019 +0200

STORM-3453: eventhubs: fix all checkstyle warnings
---
 external/storm-eventhubs/pom.xml  |  2 +-
 .../java/org/apache/storm/eventhubs/bolt/EventHubBolt.java|  1 -
 .../org/apache/storm/eventhubs/bolt/IEventDataFormat.java |  2 +-
 .../apache/storm/eventhubs/spout/BinaryEventDataScheme.java   |  2 +-
 .../org/apache/storm/eventhubs/spout/EventDataScheme.java | 11 +--
 .../java/org/apache/storm/eventhubs/spout/EventDataWrap.java  |  3 +--
 .../apache/storm/eventhubs/spout/EventHubReceiverImpl.java| 10 +-
 .../java/org/apache/storm/eventhubs/spout/EventHubSpout.java  |  8 +---
 .../storm/eventhubs/spout/IEventHubReceiverFactory.java   |  2 +-
 .../storm/eventhubs/spout/IPartitionManagerFactory.java   |  2 +-
 .../apache/storm/eventhubs/spout/SimplePartitionManager.java  |  2 +-
 .../apache/storm/eventhubs/spout/StringEventDataScheme.java   |  9 -
 .../storm/eventhubs/trident/ITridentPartitionManager.java |  2 +-
 .../storm/eventhubs/trident/OpaqueTridentEventHubEmitter.java |  2 +-
 .../storm/eventhubs/trident/OpaqueTridentEventHubSpout.java   |  2 +-
 .../java/org/apache/storm/eventhubs/trident/Partition.java|  2 +-
 .../trident/TransactionalTridentEventHubEmitter.java  |  2 --
 .../eventhubs/trident/TransactionalTridentEventHubSpout.java  |  2 +-
 18 files changed, 27 insertions(+), 39 deletions(-)

diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml
index a6a1cef..8dec463 100755
--- a/external/storm-eventhubs/pom.xml
+++ b/external/storm-eventhubs/pom.xml
@@ -53,7 +53,7 @@
 maven-checkstyle-plugin
 
 
-45
+0
 
 
 
diff --git 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
index 85ffd03..4c1e068 100755
--- 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
+++ 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
@@ -18,7 +18,6 @@
 
 package org.apache.storm.eventhubs.bolt;
 
-
 import com.microsoft.azure.eventhubs.EventData;
 import com.microsoft.azure.eventhubs.EventHubClient;
 import com.microsoft.azure.eventhubs.PartitionSender;
diff --git 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/IEventDataFormat.java
 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/IEventDataFormat.java
index ec09460..08f8e63 100644
--- 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/IEventDataFormat.java
+++ 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/IEventDataFormat.java
@@ -22,7 +22,7 @@ import java.io.Serializable;
 import org.apache.storm.tuple.Tuple;
 
 /**
- * Serialize a tuple to a byte array to be sent to EventHubs
+ * Serialize a tuple to a byte array to be sent to EventHubs.
  */
 public interface IEventDataFormat extends Serializable {
 public byte[] serialize(Tuple tuple);
diff --git 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
index 4c8e0a2..40db61a 100644
--- 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
+++ 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
 /**
  * An Event Data Scheme which deserializes message payload into the raw bytes.
  *
- * The resulting tuple would contain three items, the first being the message 
bytes, and the second a map of properties that include
+ * The resulting tuple would contain three items, the first being the 
message bytes, and the second a map of properties that include
  * metadata, which can be used to determine who processes the message, and how 
it is processed.The third is the system properties which
  * exposes information like enqueue-time, offset and sequence number
  */
diff --git 
a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/EventDataScheme.java
 
b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout

[storm] branch master updated: STORM-3452: blobstore-migration: fix all checkstyle warnings

2019-07-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 83be1b9  STORM-3452: blobstore-migration: fix all checkstyle warnings
 new 3cb335e  Merge pull request #3070 from 
krichter722/checkstyle-blobstore-migration
83be1b9 is described below

commit 83be1b9c015255162df41085b138c36921350ded
Author: Karl-Philipp Richter 
AuthorDate: Thu Jul 4 21:06:50 2019 +0200

STORM-3452: blobstore-migration: fix all checkstyle warnings
---
 external/storm-blobstore-migration/pom.xml |  2 +-
 .../java/org/apache/storm/blobstore/ListHDFS.java  | 20 ++
 .../org/apache/storm/blobstore/ListLocalFs.java|  3 +-
 .../org/apache/storm/blobstore/MigrateBlobs.java   | 44 +-
 .../org/apache/storm/blobstore/MigratorMain.java   | 20 +-
 5 files changed, 50 insertions(+), 39 deletions(-)

diff --git a/external/storm-blobstore-migration/pom.xml 
b/external/storm-blobstore-migration/pom.xml
index 69d4c5e..d7c2eb2 100644
--- a/external/storm-blobstore-migration/pom.xml
+++ b/external/storm-blobstore-migration/pom.xml
@@ -129,7 +129,7 @@ limitations under the License.
 maven-checkstyle-plugin
 
 
-56
+0
 
 
 
diff --git 
a/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListHDFS.java
 
b/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListHDFS.java
index 3f4566e..96e7038 100644
--- 
a/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListHDFS.java
+++ 
b/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListHDFS.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.blobstore;
 
 import java.util.Map;
@@ -27,14 +28,17 @@ import org.apache.storm.hdfs.blobstore.HdfsBlobStore;
 import org.apache.storm.hdfs.blobstore.HdfsClientBlobStore;
 import org.apache.storm.utils.Utils;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class ListHDFS {
 
 public static void main(String[] args) throws Exception {
-if(args.length < 1) {
+if (args.length < 1) {
 System.out.println("Need at least 1 argument 
(hdfs_blobstore_path), but have " + Integer.toString(args.length));
 System.out.println("listHDFS  
 ");
 System.out.println("Lists blobs in HdfsBlobStore");
-System.out.println("Example: listHDFS 
'hdfs://some-hdfs-namenode:8080/srv/storm/my-storm-blobstore' 
'stormUser/my-nimbus-host.example@storm.example.com' 
'/srv/my-keytab/stormUser.kt'");
+System.out.println("Example: listHDFS "
++ 
"'hdfs://some-hdfs-namenode:8080/srv/storm/my-storm-blobstore' "
++ 
"'stormUser/my-nimbus-host.example@storm.example.com' 
'/srv/my-keytab/stormUser.kt'");
 System.exit(1);
 }
 
@@ -43,13 +47,13 @@ public class ListHDFS {
 
 hdfsConf.put(Config.BLOBSTORE_DIR, hdfsBlobstorePath);
 hdfsConf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN, 
"org.apache.storm.security.auth.DefaultPrincipalToLocal");
-if(args.length >= 2) {
-   System.out.println("SETTING HDFS PRINCIPAL!");
-   hdfsConf.put(Config.BLOBSTORE_HDFS_PRINCIPAL, args[1]);
+if (args.length >= 2) {
+System.out.println("SETTING HDFS PRINCIPAL!");
+hdfsConf.put(Config.BLOBSTORE_HDFS_PRINCIPAL, args[1]);
 }
-if(args.length >= 3) {
-   System.out.println("SETTING HDFS KEYTAB!");
-   hdfsConf.put(Config.BLOBSTORE_HDFS_KEYTAB, args[2]);
+if (args.length >= 3) {
+System.out.println("SETTING HDFS KEYTAB!");
+hdfsConf.put(Config.BLOBSTORE_HDFS_KEYTAB, args[2]);
 }
 
 /* CREATE THE BLOBSTORES */
diff --git 
a/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListLocalFs.java
 
b/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListLocalFs.java
index 35ff33f..aefddb8 100644
--- 
a/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListLocalFs.java
+++ 
b/external/storm-blobstore-migration/src/main/java/org/apache/storm/blobstore/ListLocalFs.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.blobstore;
 
 import java.util.Map;
@@ -30,7 +31,7 @@ public class ListLo

[storm] branch master updated: STORM-3449: autocreds: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new f5aecc1  STORM-3449: autocreds: fix all checkstyle warnings
 new 7ca1ffb  Merge pull request #3065 from krichter722/checkstyle-autocreds
f5aecc1 is described below

commit f5aecc1b5d5e15c0c128f600c5a399eb0b0e8fc4
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 21:54:37 2019 +0200

STORM-3449: autocreds: fix all checkstyle warnings
---
 external/storm-autocreds/pom.xml   |  2 +-
 .../storm/common/AbstractHadoopAutoCreds.java  | 29 +++
 .../AbstractHadoopNimbusPluginAutoCreds.java   | 25 +++---
 .../apache/storm/common/CredentialKeyProvider.java | 13 +--
 .../apache/storm/common/HadoopCredentialUtil.java  | 85 ++-
 .../org/apache/storm/hbase/security/AutoHBase.java |  4 +-
 .../storm/hbase/security/AutoHBaseCommand.java | 71 
 .../storm/hbase/security/AutoHBaseNimbus.java  | 34 
 .../storm/hbase/security/HBaseSecurityUtil.java| 14 ++--
 .../org/apache/storm/hdfs/security/AutoHDFS.java   |  5 +-
 .../storm/hdfs/security/AutoHDFSCommand.java   | 58 ++---
 .../apache/storm/hdfs/security/AutoHDFSNimbus.java | 57 +++--
 .../storm/hdfs/security/HdfsSecurityUtil.java  | 24 +++---
 .../org/apache/storm/hive/security/AutoHive.java   |  6 +-
 .../storm/hive/security/AutoHiveCommand.java   | 62 +++---
 .../apache/storm/hive/security/AutoHiveNimbus.java | 97 --
 .../storm/hive/security/HiveSecurityUtil.java  | 13 +--
 17 files changed, 312 insertions(+), 287 deletions(-)

diff --git a/external/storm-autocreds/pom.xml b/external/storm-autocreds/pom.xml
index a8ac75e..87b2a2f 100644
--- a/external/storm-autocreds/pom.xml
+++ b/external/storm-autocreds/pom.xml
@@ -207,7 +207,7 @@
 maven-checkstyle-plugin
 
 
-249
+0
 
 
 
diff --git 
a/external/storm-autocreds/src/main/java/org/apache/storm/common/AbstractHadoopAutoCreds.java
 
b/external/storm-autocreds/src/main/java/org/apache/storm/common/AbstractHadoopAutoCreds.java
index 569f5af..7800401 100644
--- 
a/external/storm-autocreds/src/main/java/org/apache/storm/common/AbstractHadoopAutoCreds.java
+++ 
b/external/storm-autocreds/src/main/java/org/apache/storm/common/AbstractHadoopAutoCreds.java
@@ -15,8 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.common;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.security.auth.Subject;
+import javax.xml.bind.DatatypeConverter;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.security.Credentials;
@@ -27,15 +37,6 @@ import org.apache.storm.security.auth.IAutoCredentials;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.security.auth.Subject;
-import javax.xml.bind.DatatypeConverter;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 /**
  * The base class that for auto credential plugins that abstracts out some of 
the common functionality.
  */
@@ -62,7 +63,7 @@ public abstract class AbstractHadoopAutoCreds implements 
IAutoCredentials, Crede
 @Override
 public void populateSubject(Subject subject, Map 
credentials) {
 addCredentialToSubject(subject, credentials);
-addTokensToUGI(subject);
+addTokensToUgi(subject);
 }
 
 /**
@@ -71,7 +72,7 @@ public abstract class AbstractHadoopAutoCreds implements 
IAutoCredentials, Crede
 @Override
 public void updateSubject(Subject subject, Map 
credentials) {
 addCredentialToSubject(subject, credentials);
-addTokensToUGI(subject);
+addTokensToUgi(subject);
 }
 
 public Set> getCredentials(Map 
credentials) {
@@ -79,14 +80,14 @@ public abstract class AbstractHadoopAutoCreds implements 
IAutoCredentials, Crede
 }
 
 /**
- * Prepare the plugin
+ * Prepare the plugin.
  *
  * @param topoConf the topology conf
  */
 protected abstract void doPrepare(Map topoConf);
 
 /**
- * The lookup key for the config key string
+ * The lookup key for the config key string.
  *
  * @return the config key string
  */
@@ -104,7 +105,7 @@ public abstract class AbstractHadoopAutoCreds implements 
IAutoCredentials, Crede
 }
 }
 
-private void addTokensToUGI(Subject subject) {
+private void addTokensToUgi(Subject subj

[storm] branch master updated: STORM-3418: Fix checkstyle violations in storm-hdfs-blobstore

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 6549a51  STORM-3418: Fix checkstyle violations in storm-hdfs-blobstore
 new bba702e  Merge pull request #3042 from 
krichter722/checkstyle-hdfs-blobstore
6549a51 is described below

commit 6549a5107137a79585f5b81ab61e2b7b5df92aee
Author: Karl-Philipp Richter 
AuthorDate: Sat Jun 15 20:32:25 2019 +0200

STORM-3418: Fix checkstyle violations in storm-hdfs-blobstore
---
 external/storm-hdfs-blobstore/pom.xml  |   2 +-
 .../apache/storm/hdfs/blobstore/HdfsBlobStore.java |  45 
 .../storm/hdfs/blobstore/HdfsBlobStoreFile.java| 122 +++--
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java| 102 -
 .../storm/hdfs/blobstore/HdfsClientBlobStore.java  |  51 -
 .../apache/storm/blobstore/ClientBlobStore.java|   1 -
 .../storm/blobstore/ClientBlobStoreTest.java   |   1 -
 7 files changed, 164 insertions(+), 160 deletions(-)

diff --git a/external/storm-hdfs-blobstore/pom.xml 
b/external/storm-hdfs-blobstore/pom.xml
index a369ebe..e2b948b 100644
--- a/external/storm-hdfs-blobstore/pom.xml
+++ b/external/storm-hdfs-blobstore/pom.xml
@@ -255,7 +255,7 @@
 maven-checkstyle-plugin
 
 
-80
+0
 
 
 
diff --git 
a/external/storm-hdfs-blobstore/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
 
b/external/storm-hdfs-blobstore/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
index 814c5d4..da918c4 100644
--- 
a/external/storm-hdfs-blobstore/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
+++ 
b/external/storm-hdfs-blobstore/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
@@ -18,6 +18,10 @@
 
 package org.apache.storm.hdfs.blobstore;
 
+import static org.apache.storm.blobstore.BlobStoreAclHandler.ADMIN;
+import static org.apache.storm.blobstore.BlobStoreAclHandler.READ;
+import static org.apache.storm.blobstore.BlobStoreAclHandler.WRITE;
+
 import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -50,20 +54,18 @@ import org.apache.storm.utils.WrappedKeyNotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.storm.blobstore.BlobStoreAclHandler.*;
-
 /**
  * Provides a HDFS file system backed blob store implementation.
  * Note that this provides an api for having HDFS be the backing store for the 
blobstore,
  * it is not a service/daemon.
  *
- * We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS configuration. 
NIMBUS_ADMINS are given READ, WRITE and ADMIN
+ * 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
+ * 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 hdfs blob store
+ * For hdfs blob store
  * 1. The USER interacts with nimbus to upload and access blobs through 
NimbusBlobStore Client API. Here, unlike
  * local blob store which stores the blobs locally, the nimbus talks to HDFS 
to upload the blobs.
  * 2. The USER sets the ACLs, and the blob access is validated against these 
ACLs.
@@ -191,23 +193,23 @@ public class HdfsBlobStore extends BlobStore {
 if (hbs.exists(DATA_PREFIX + key)) {
 throw new WrappedKeyAlreadyExistsException(key);
 }
-BlobStoreFileOutputStream mOut = null;
+BlobStoreFileOutputStream outputStream = null;
 try {
 BlobStoreFile metaFile = hbs.write(META_PREFIX + key, true);
 metaFile.setMetadata(meta);
-mOut = new BlobStoreFileOutputStream(metaFile);
-mOut.write(Utils.thriftSerialize(meta));
-mOut.close();
-mOut = null;
+outputStream = new BlobStoreFileOutputStream(metaFile);
+outputStream.write(Utils.thriftSerialize(meta));
+outputStream.close();
+outputStream = null;
 BlobStoreFile dataFile = hbs.write(DATA_PREFIX + key, true);
 dataFile.setMetadata(meta);
 return new BlobStoreFileOutputStream(dataFile);
 } catch (IOException e) {
 throw new RuntimeException(e);
 } finally {
-if (mOut != null) {
+if (outputStream != null) {
 try

[storm] branch master updated: STORM-3446: storm-maven-plugins: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b1ed064  STORM-3446: storm-maven-plugins: fix all checkstyle warnings
 new ace6289  Merge pull request #3061 from 
krichter722/checkstyle-maven-plugins
b1ed064 is described below

commit b1ed06452d809cc2402f77fc3d29c9dd462195d9
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 20:32:20 2019 +0200

STORM-3446: storm-maven-plugins: fix all checkstyle warnings
---
 storm-buildtools/storm-maven-plugins/pom.xml   |  2 +-
 .../maven/plugin/versioninfo/VersionInfoMojo.java  | 33 +++---
 2 files changed, 24 insertions(+), 11 deletions(-)

diff --git a/storm-buildtools/storm-maven-plugins/pom.xml 
b/storm-buildtools/storm-maven-plugins/pom.xml
index c5b1b59..9a8b946 100644
--- a/storm-buildtools/storm-maven-plugins/pom.xml
+++ b/storm-buildtools/storm-maven-plugins/pom.xml
@@ -75,7 +75,7 @@
 maven-checkstyle-plugin
 
 
-  11
+  0
 
   
   
diff --git 
a/storm-buildtools/storm-maven-plugins/src/main/java/org/apache/storm/maven/plugin/versioninfo/VersionInfoMojo.java
 
b/storm-buildtools/storm-maven-plugins/src/main/java/org/apache/storm/maven/plugin/versioninfo/VersionInfoMojo.java
index 9eb0298..eb1a3ae 100644
--- 
a/storm-buildtools/storm-maven-plugins/src/main/java/org/apache/storm/maven/plugin/versioninfo/VersionInfoMojo.java
+++ 
b/storm-buildtools/storm-maven-plugins/src/main/java/org/apache/storm/maven/plugin/versioninfo/VersionInfoMojo.java
@@ -89,7 +89,7 @@ public class VersionInfoMojo extends AbstractMojo {
 @Override
 public void execute() throws MojoExecutionException {
 try {
-SCM scm = determineSCM();
+SCM scm = determineScm();
 project.getProperties().setProperty(buildTimeProperty, 
getBuildTime());
 project.getProperties().setProperty(scmUriProperty, 
getSCMUri(scm));
 project.getProperties().setProperty(scmBranchProperty, 
getSCMBranch(scm));
@@ -113,7 +113,7 @@ public class VersionInfoMojo extends AbstractMojo {
  * @return SCM in use for this build
  * @throws Exception if any error occurs attempting to determine SCM
  */
-private SCM determineSCM() throws Exception {
+private SCM determineScm() throws Exception {
 CommandExec exec = new CommandExec(this);
 SCM scm = SCM.NONE;
 scmOut = new ArrayList();
@@ -172,6 +172,7 @@ public class VersionInfoMojo extends AbstractMojo {
 return res;
 }
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 private String getSCMUri(SCM scm) {
 String uri = "Unknown";
 switch (scm) {
@@ -193,10 +194,14 @@ public class VersionInfoMojo extends AbstractMojo {
 }
 }
 break;
+default:
+throw new IllegalArgumentException(String.format("SCM %s is 
not supported",
+scm));
 }
 return uri.trim();
 }
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 private String getSCMCommit(SCM scm) {
 String commit = "Unknown";
 switch (scm) {
@@ -216,10 +221,14 @@ public class VersionInfoMojo extends AbstractMojo {
 }
 }
 break;
+default:
+throw new IllegalArgumentException(String.format("SCM %s is 
not supported",
+scm));
 }
 return commit.trim();
 }
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 private String getSCMBranch(SCM scm) {
 String branch = "Unknown";
 switch (scm) {
@@ -240,6 +249,9 @@ public class VersionInfoMojo extends AbstractMojo {
 }
 }
 break;
+default:
+throw new IllegalArgumentException(String.format("SCM %s is 
not supported",
+scm));
 }
 return branch.trim();
 }
@@ -262,14 +274,6 @@ public class VersionInfoMojo extends AbstractMojo {
 return md5.digest();
 }
 
-private String byteArrayToString(byte[] array) {
-StringBuilder sb = new StringBuilder();
-for (byte b : array) {
-sb.append(Integer.toHexString(0xff & b));
-}
-return sb.toString();
-}
-
 private String computeMD5() throws Exception {
 List files = convertFileSetToFiles(source);
 // File order of MD5 calculation is significant. Sorting is done on
@@ -291,6 +295,15 @@ public class VersionInfoMojo extends AbstractMojo {
 return md5str;
 }
 
+private String byteArrayToString

[storm] branch master updated: STORM-3447: jms: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 82b8bdf  STORM-3447: jms: fix all checkstyle warnings
 new 0b3ca35  Merge pull request #3062 from krichter722/checkstyle-jms
82b8bdf is described below

commit 82b8bdf8908de117fef0f5178d11325e70b42221
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 20:46:09 2019 +0200

STORM-3447: jms: fix all checkstyle warnings
---
 external/storm-jms/pom.xml |  2 +-
 .../org/apache/storm/jms/JmsMessageProducer.java   |  8 +--
 .../java/org/apache/storm/jms/JmsProvider.java |  6 +-
 .../org/apache/storm/jms/JmsTupleProducer.java |  9 ++-
 .../java/org/apache/storm/jms/bolt/JmsBolt.java| 47 +
 .../org/apache/storm/jms/spout/JmsMessageID.java   | 15 +++--
 .../java/org/apache/storm/jms/spout/JmsSpout.java  |  2 +-
 .../org/apache/storm/jms/trident/JmsState.java |  4 +-
 .../apache/storm/jms/trident/JmsStateFactory.java  |  2 +-
 .../apache/storm/jms/trident/TridentJmsSpout.java  | 78 ++
 10 files changed, 69 insertions(+), 104 deletions(-)

diff --git a/external/storm-jms/pom.xml b/external/storm-jms/pom.xml
index 984d8cf..ceb4efb 100644
--- a/external/storm-jms/pom.xml
+++ b/external/storm-jms/pom.xml
@@ -81,7 +81,7 @@
 maven-checkstyle-plugin
 
 
-63
+0
 
 
 
diff --git 
a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsMessageProducer.java 
b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsMessageProducer.java
index 671cdd9..1dcd608 100644
--- 
a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsMessageProducer.java
+++ 
b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsMessageProducer.java
@@ -22,18 +22,12 @@ import org.apache.storm.tuple.ITuple;
  * JmsMessageProducer implementations are responsible for translating
  * a org.apache.storm.tuple.Values instance into a
  * javax.jms.Message object.
- * 
  */
 public interface JmsMessageProducer extends Serializable {
 
 /**
  * Translate a org.apache.storm.tuple.Tuple object
- * to a javax.jms.Messagejavax.jms.Message object.
  */
 public Message toMessage(Session session, ITuple input) throws 
JMSException;
 }
diff --git 
a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsProvider.java 
b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsProvider.java
index b8dde44..324ff9d 100644
--- a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsProvider.java
+++ b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsProvider.java
@@ -24,19 +24,15 @@ import javax.jms.Destination;
  */
 public interface JmsProvider extends Serializable {
 /**
- * Provides the JMS ConnectionFactory
+ * Provides the JMS ConnectionFactory.
  *
  * @return the connection factory
- * @throws Exception
  */
 public ConnectionFactory connectionFactory() throws Exception;
 
 /**
  * Provides the Destination (topic or queue) from which the
  * JmsSpout will receive messages.
- *
- * @return
- * @throws Exception
  */
 public Destination destination() throws Exception;
 }
diff --git 
a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsTupleProducer.java 
b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsTupleProducer.java
index 4457f5a..44adf99 100644
--- 
a/external/storm-jms/src/main/java/org/apache/storm/jms/JmsTupleProducer.java
+++ 
b/external/storm-jms/src/main/java/org/apache/storm/jms/JmsTupleProducer.java
@@ -21,11 +21,11 @@ import org.apache.storm.tuple.Values;
 /**
  * Interface to define classes that can produce a Storm Values 
objects
  * from a javax.jms.Message object>.
- * 
- * Implementations are also responsible for declaring the output
+ *
+ * Implementations are also responsible for declaring the output
  * fields they produce.
- * 
- * If for some reason the implementation can't process a message
+ *
+ * If for some reason the implementation can't process a message
  * (for example if it received a javax.jms.ObjectMessage
  * when it was expecting a javax.jms.TextMessage it should
  * return null to indicate to the JmsSpout that
@@ -38,7 +38,6 @@ public interface JmsTupleProducer extends Serializable {
  *
  * @param msg - the JMS message
  * @return the Values tuple, or null if the message couldn't be processed.
- * @throws JMSException
  */
 Values toTuple(Message msg) throws JMSException;
 
diff --git 
a/external/storm-jms/src/main/java/org/apache/storm/jms/bolt/JmsBolt.java 
b/external/storm-jms/src/main/java/org/apache/storm/jms/bolt/JmsBolt.java
index 0b461a1..362c3a6 100644
--- a/external/storm-jms/src/main/java/org/apache/storm/jms/b

[storm] branch master updated: STORM-3450: perf: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 7bf7268  STORM-3450: perf: fix all checkstyle warnings
 new 67ae024  Merge pull request #3066 from krichter722/checkstyle-perf
7bf7268 is described below

commit 7bf7268e25a627882ddccc68fdf58df9889011a5
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 22:18:27 2019 +0200

STORM-3450: perf: fix all checkstyle warnings
---
 examples/storm-perf/pom.xml|   2 +-
 .../storm/perf/ConstSpoutIdBoltNullBoltTopo.java   |   2 +-
 .../apache/storm/perf/ConstSpoutNullBoltTopo.java  |  13 +-
 .../org/apache/storm/perf/ConstSpoutOnlyTopo.java  |  11 +-
 .../apache/storm/perf/FileReadWordCountTopo.java   |   6 +-
 .../apache/storm/perf/HdfsSpoutNullBoltTopo.java   |  10 +-
 .../org/apache/storm/perf/KafkaClientHdfsTopo.java |  16 +-
 .../org/apache/storm/perf/LowThroughputTopo.java   |   1 -
 .../apache/storm/perf/StrGenSpoutHdfsBoltTopo.java |  20 +-
 .../org/apache/storm/perf/ThroughputMeter.java |   1 +
 .../org/apache/storm/perf/bolt/DevNullBolt.java|   4 +-
 .../org/apache/storm/perf/queuetest/Acker.java |  62 ++
 .../storm/perf/queuetest/AckingProducer.java   |  62 ++
 .../org/apache/storm/perf/queuetest/Consumer.java  |  63 ++
 .../org/apache/storm/perf/queuetest/Forwarder.java |  70 ++
 .../perf/{ => queuetest}/JCQueuePerfTest.java  | 239 +
 .../org/apache/storm/perf/queuetest/MyThread.java  |  36 
 .../org/apache/storm/perf/queuetest/Producer.java  |  44 
 .../org/apache/storm/perf/queuetest/Producer2.java |  50 +
 .../org/apache/storm/perf/spout/WordGenSpout.java  |   3 +-
 .../java/org/apache/storm/perf/toolstest/Cons.java |  61 ++
 .../perf/{ => toolstest}/JCToolsPerfTest.java  | 108 +-
 .../org/apache/storm/perf/toolstest/MyThd.java |  37 
 .../java/org/apache/storm/perf/toolstest/Prod.java |  46 
 .../org/apache/storm/perf/toolstest/Prod2.java |  46 
 .../java/org/apache/storm/perf/utils/Helper.java   |   2 +-
 .../org/apache/storm/perf/utils/MetricsSample.java |   4 +-
 27 files changed, 627 insertions(+), 392 deletions(-)

diff --git a/examples/storm-perf/pom.xml b/examples/storm-perf/pom.xml
index 527ef49..07c4b17 100644
--- a/examples/storm-perf/pom.xml
+++ b/examples/storm-perf/pom.xml
@@ -92,7 +92,7 @@
 maven-checkstyle-plugin
 
 
-65
+0
 
 
 
diff --git 
a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
 
b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
index a65efe6..c83763d 100644
--- 
a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
+++ 
b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
@@ -30,7 +30,7 @@ import org.apache.storm.utils.Utils;
 
 /**
  * ConstSpout -> IdBolt -> DevNullBolt This topology measures speed of 
messaging between spouts->bolt  and  bolt->bolt ConstSpout :
- * Continuously emits a constant string IdBolt : clones and emits input tuples 
DevNullBolt : discards incoming tuples
+ * Continuously emits a constant string IdBolt : clones and emits input tuples 
DevNullBolt : discards incoming tuples.
  */
 public class ConstSpoutIdBoltNullBoltTopo {
 
diff --git 
a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
 
b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
index 9683e08..63ef51b 100755
--- 
a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
+++ 
b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
@@ -28,12 +28,13 @@ import org.apache.storm.topology.BoltDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.utils.Utils;
 
-/***
- *  This topo helps measure the messaging peak throughput between a spout and 
a bolt.
- *  Spout generates a stream of a fixed string.
- *  Bolt will simply ack and discard the tuple received
+/**
+ * This topo helps measure the messaging peak throughput between a spout and a 
bolt.
+ *
+ * Spout generates a stream of a fixed string.
+ *
+ * Bolt will simply ack and discard the tuple received.
  */
-
 public class ConstSpoutNullBoltTopo {
 
 public static final String TOPOLOGY_NAME = "ConstSpoutNullBoltTopo";
@@ -79,7 +80,7 @@ public class ConstSpoutNullBoltTopo {
 }
 
 /**
- * ConstSpout -> DevNullBolt with configurable grouping (default 
localOrShuffle)
+ * ConstSpout -> DevNullBolt with configurable grouping (default 
localOrShuffle).
  */
 pub

[storm] branch master updated: STORM-3448: core: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new e3825f3  STORM-3448: core: fix all checkstyle warnings
 new 3ecaf62  Merge pull request #3063 from krichter722/checkstyle-core
e3825f3 is described below

commit e3825f338347a027a0cab190c79bbb3c4f3a0892
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 20:57:32 2019 +0200

STORM-3448: core: fix all checkstyle warnings
---
 storm-core/pom.xml |  2 +-
 .../org/apache/storm/command/AdminCommands.java| 19 +++---
 .../src/jvm/org/apache/storm/command/CLI.java  |  2 +
 .../org/apache/storm/command/ListTopologies.java   |  9 ++-
 .../src/jvm/org/apache/storm/command/Monitor.java  | 19 +++---
 .../jvm/org/apache/storm/command/Rebalance.java|  9 ++-
 .../org/apache/storm/command/ShellSubmission.java  |  6 +-
 .../org/apache/storm/planner/CompoundSpout.java|  6 +-
 .../jvm/org/apache/storm/planner/CompoundTask.java |  6 +-
 .../src/jvm/org/apache/storm/utils/Monitor.java| 76 --
 .../org/apache/storm/utils/TopologySpoutLag.java   | 18 ++---
 11 files changed, 88 insertions(+), 84 deletions(-)

diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 2bebab4..71d220f 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -303,7 +303,7 @@
 maven-checkstyle-plugin
 
 
-73
+0
 
 
 
diff --git a/storm-core/src/jvm/org/apache/storm/command/AdminCommands.java 
b/storm-core/src/jvm/org/apache/storm/command/AdminCommands.java
index 564a01c..8076b74 100644
--- a/storm-core/src/jvm/org/apache/storm/command/AdminCommands.java
+++ b/storm-core/src/jvm/org/apache/storm/command/AdminCommands.java
@@ -72,7 +72,8 @@ public class AdminCommands {
 @Override
 public void run(String[] args, Map conf, String 
command) throws Exception {
 try (BlobStore nimbusBlobStore = 
ServerUtils.getNimbusBlobStore(conf, NimbusInfo.fromConf(conf), null)) {
-IStormClusterState stormClusterState = 
ClusterUtils.mkStormClusterState(conf, new 
ClusterStateContext(DaemonType.NIMBUS, conf));
+IStormClusterState stormClusterState = 
ClusterUtils.mkStormClusterState(conf,
+new ClusterStateContext(DaemonType.NIMBUS, conf));
 
 Set blobStoreTopologyIds = 
nimbusBlobStore.filterAndListKeys(key -> ConfigUtils.getIdFromBlobKey(key));
 Set activeTopologyIds = new 
HashSet<>(stormClusterState.activeStorms());
@@ -126,14 +127,6 @@ public class AdminCommands {
 return builder.toString();
 }
 
-private static void println(StringBuilder out, int depth, Object value) {
-for (int i = 0; i < depth; i++) {
-out.append("\t");
-}
-out.append(value);
-out.append("\n");
-}
-
 private static void prettyPrint(TBase value, int depth, StringBuilder out) 
{
 if (value == null) {
 println(out, depth,"null");
@@ -144,6 +137,14 @@ public class AdminCommands {
 println(out, depth, "}");
 }
 
+private static void println(StringBuilder out, int depth, Object value) {
+for (int i = 0; i < depth; i++) {
+out.append("\t");
+}
+out.append(value);
+out.append("\n");
+}
+
 private static void prettyPrintFields(TBase value, int depth, 
StringBuilder out) {
 for (Map.Entry entry : 
FieldMetaData.getStructMetaDataMap(value.getClass()).entrySet()) {
 TFieldIdEnum key = entry.getKey();
diff --git a/storm-core/src/jvm/org/apache/storm/command/CLI.java 
b/storm-core/src/jvm/org/apache/storm/command/CLI.java
index 4510d29..cc5c5dc 100644
--- a/storm-core/src/jvm/org/apache/storm/command/CLI.java
+++ b/storm-core/src/jvm/org/apache/storm/command/CLI.java
@@ -23,6 +23,7 @@ import org.apache.commons.cli.Options;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class CLI {
 /**
  * Parse function to return an Integer.
@@ -250,6 +251,7 @@ public class CLI {
 }
 }
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public static class CLIBuilder {
 private final ArrayList opts = new ArrayList<>();
 private final ArrayList args = new ArrayList<>();
diff --git a/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java 
b/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
index 13d9691..09068d3 100644
--- a/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
+++ b/storm-core/src/jvm/org/apache/storm/c

[storm] branch master updated: STORM-3443: hive: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4bae551  STORM-3443: hive: fix all checkstyle warnings
 new 04af947  Merge pull request #3058 from krichter722/checkstyle-hive
4bae551 is described below

commit 4bae551127c4e34eab49a352924bbc9fbbe6b83b
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 20:00:41 2019 +0200

STORM-3443: hive: fix all checkstyle warnings
---
 external/storm-hive/pom.xml|  2 +-
 .../java/org/apache/storm/hive/bolt/HiveBolt.java  | 14 
 .../bolt/mapper/DelimitedRecordHiveMapper.java | 25 +++---
 .../apache/storm/hive/bolt/mapper/HiveMapper.java  | 24 +++---
 .../hive/bolt/mapper/JsonRecordHiveMapper.java | 23 +++--
 .../org/apache/storm/hive/common/HiveOptions.java  |  6 ++--
 .../org/apache/storm/hive/common/HiveUtils.java|  8 ++---
 .../org/apache/storm/hive/common/HiveWriter.java   | 38 +-
 .../org/apache/storm/hive/trident/HiveState.java   | 14 
 9 files changed, 67 insertions(+), 87 deletions(-)

diff --git a/external/storm-hive/pom.xml b/external/storm-hive/pom.xml
index 94b73db..4ca6f0b 100644
--- a/external/storm-hive/pom.xml
+++ b/external/storm-hive/pom.xml
@@ -246,7 +246,7 @@
 maven-checkstyle-plugin
 
 
-  58
+  0
 
   
   
diff --git 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
index 180f41b..db373e5 100644
--- a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
+++ b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
@@ -129,15 +129,15 @@ public class HiveBolt extends BaseRichBolt {
 HiveWriter w = entry.getValue();
 w.flushAndClose();
 } catch (Exception ex) {
-LOG.warn("Error while closing writer to " + entry.getKey() +
- ". Exception follows.", ex);
+LOG.warn("Error while closing writer to " + entry.getKey() + 
". Exception follows.",
+ex);
 if (ex instanceof InterruptedException) {
 Thread.currentThread().interrupt();
 }
 }
 }
 
-ExecutorService toShutdown[] = { callTimeoutPool };
+ExecutorService[] toShutdown = { callTimeoutPool };
 for (ExecutorService execService : toShutdown) {
 execService.shutdown();
 try {
@@ -214,7 +214,7 @@ public class HiveBolt extends BaseRichBolt {
 }
 
 /**
- * Abort current Txn on all writers
+ * Abort current Txn on all writers.
  */
 private void abortAllWriters() throws InterruptedException, 
StreamingException, HiveWriter.TxnBatchFailure {
 for (Entry entry : allWriters.entrySet()) {
@@ -227,7 +227,7 @@ public class HiveBolt extends BaseRichBolt {
 }
 
 /**
- * Closes all writers and remove them from cache
+ * Closes all writers and remove them from cache.
  */
 private void closeAllWriters() {
 //1) Retire writers
@@ -269,7 +269,7 @@ public class HiveBolt extends BaseRichBolt {
 }
 
 /**
- * Locate writer that has not been used for longest time and retire it
+ * Locate writer that has not been used for longest time and retire it.
  */
 private void retireEldestWriter() {
 LOG.info("Attempting close eldest writers");
@@ -295,7 +295,7 @@ public class HiveBolt extends BaseRichBolt {
 }
 
 /**
- * Locate all writers past idle timeout and retire them
+ * Locate all writers past idle timeout and retire them.
  * @return number of writers retired
  */
 private int retireIdleWriters() {
diff --git 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
index f6e3612..fbbc4cc 100644
--- 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
+++ 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
@@ -12,7 +12,6 @@
 
 package org.apache.storm.hive.bolt.mapper;
 
-
 import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
@@ -94,18 +93,6 @@ public class DelimitedRecordHiveMapper implements HiveMapper 
{
 }
 
 @Override
-public byte[] mapRecord(Tuple tuple) {
-StringBuilder builder = new StringBuilder();
-if (this.columnFields != null) {
-for (String field : th

[storm] branch master updated: STORM-3441: hbase: fix all checkstyle warnings

2019-07-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 905007f  STORM-3441: hbase: fix all checkstyle warnings
 new 1389492  Merge pull request #3057 from krichter722/checkstyle-hbase
905007f is described below

commit 905007fb1db4f18158c85b3f92211be7ab73138b
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 22:43:40 2019 +0200

STORM-3441: hbase: fix all checkstyle warnings
---
 external/storm-hbase/pom.xml   |  2 +-
 .../apache/storm/hbase/bolt/AbstractHBaseBolt.java |  1 +
 .../org/apache/storm/hbase/bolt/HBaseBolt.java |  5 +-
 .../apache/storm/hbase/bolt/HBaseLookupBolt.java   |  6 +--
 .../storm/hbase/bolt/mapper/HBaseMapper.java   |  8 +---
 .../hbase/bolt/mapper/HBaseProjectionCriteria.java |  9 
 .../storm/hbase/bolt/mapper/HBaseValueMapper.java  | 12 ++---
 .../storm/hbase/bolt/mapper/SimpleHBaseMapper.java | 10 ++--
 .../org/apache/storm/hbase/common/ColumnList.java  | 55 --
 .../java/org/apache/storm/hbase/common/Utils.java  |  4 +-
 .../hbase/state/HBaseKeyValueStateIterator.java|  4 +-
 .../hbase/state/HBaseKeyValueStateProvider.java| 18 +++
 .../trident/mapper/SimpleTridentHBaseMapper.java   |  9 ++--
 .../trident/mapper/TridentHBaseMapMapper.java  |  6 ---
 .../hbase/trident/mapper/TridentHBaseMapper.java   |  7 ---
 .../storm/hbase/trident/state/HBaseQuery.java  |  1 +
 .../storm/hbase/trident/state/HBaseState.java  |  5 +-
 .../hbase/trident/state/HBaseStateFactory.java |  2 +-
 .../storm/hbase/trident/state/HBaseUpdater.java|  1 +
 19 files changed, 50 insertions(+), 115 deletions(-)

diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml
index 4a9f78f..6620f6a 100644
--- a/external/storm-hbase/pom.xml
+++ b/external/storm-hbase/pom.xml
@@ -117,7 +117,7 @@
 maven-checkstyle-plugin
 
 
-100
+0
 
 
 
diff --git 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
index bb3e93c..4d0855e 100644
--- 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
+++ 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
@@ -33,6 +33,7 @@ public abstract class AbstractHBaseBolt extends BaseRichBolt {
 private static final Logger LOG = 
LoggerFactory.getLogger(AbstractHBaseBolt.class);
 
 protected transient OutputCollector collector;
+@SuppressWarnings("checkstyle:MemberName")
 protected transient HBaseClient hBaseClient;
 protected String tableName;
 protected HBaseMapper mapper;
diff --git 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
index fa6acb5..16759ca 100644
--- 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
+++ 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
@@ -31,12 +31,14 @@ import org.slf4j.LoggerFactory;
 /**
  * Basic bolt for writing to HBase.
  *
- * Note: Each HBaseBolt defined in a topology is tied to a specific table.
+ * Note: Each HBaseBolt defined in a topology is tied to a specific table.
  */
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class HBaseBolt extends AbstractHBaseBolt {
 private static final Logger LOG = LoggerFactory.getLogger(HBaseBolt.class);
 private static final int DEFAULT_FLUSH_INTERVAL_SECS = 1;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 boolean writeToWAL = true;
 List batchMutations;
 int flushIntervalSecs = DEFAULT_FLUSH_INTERVAL_SECS;
@@ -48,6 +50,7 @@ public class HBaseBolt extends AbstractHBaseBolt {
 this.batchMutations = new LinkedList<>();
 }
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public HBaseBolt writeToWAL(boolean writeToWAL) {
 this.writeToWAL = writeToWAL;
 return this;
diff --git 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
index 538d896..85262ca 100644
--- 
a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
+++ 
b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Basic bolt for querying from HBase.
  *
- * Note: Each HBaseBolt defined in a topology is tied to a specific table.
+ * Note: Each HBaseBolt def

[storm] branch master updated: STORM-3444: redis: fix all checkstyle warnings

2019-07-03 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 3c2e03e  STORM-3444: redis: fix all checkstyle warnings
 new 8950d9b  Merge pull request #3059 from krichter722/checkstyle-redis
3c2e03e is described below

commit 3c2e03ecfd1657ab99ceea9226f31667f8fd32e8
Author: Karl-Philipp Richter 
AuthorDate: Mon Jul 1 20:18:09 2019 +0200

STORM-3444: redis: fix all checkstyle warnings
---
 external/storm-redis/pom.xml   |  2 +-
 .../apache/storm/redis/bolt/AbstractRedisBolt.java | 12 -
 .../apache/storm/redis/bolt/RedisFilterBolt.java   | 20 +++
 .../apache/storm/redis/bolt/RedisLookupBolt.java   |  8 +++---
 .../apache/storm/redis/bolt/RedisStoreBolt.java| 10 
 .../common/adapter/RedisCommandsAdapterJedis.java  | 30 +++---
 .../adapter/RedisCommandsAdapterJedisCluster.java  | 30 +++---
 .../storm/redis/common/commands/RedisCommands.java |  4 +--
 .../storm/redis/common/config/JedisPoolConfig.java | 14 +-
 .../common/container/JedisClusterContainer.java|  2 +-
 .../container/JedisCommandsInstanceContainer.java  |  2 +-
 .../redis/common/container/JedisContainer.java |  2 +-
 .../common/container/RedisClusterContainer.java|  2 +-
 .../redis/common/container/RedisContainer.java |  2 +-
 .../common/mapper/RedisDataTypeDescription.java| 12 +
 .../storm/redis/state/RedisKeyValueState.java  |  4 +--
 .../redis/state/RedisKeyValueStateProvider.java| 18 ++---
 .../trident/state/AbstractRedisStateQuerier.java   |  2 +-
 .../trident/state/AbstractRedisStateUpdater.java   |  8 +++---
 .../storm/redis/trident/state/KeyFactory.java  |  2 +-
 .../redis/trident/state/RedisClusterMapState.java  |  4 +--
 .../redis/trident/state/RedisClusterState.java | 12 -
 .../trident/state/RedisClusterStateQuerier.java|  2 +-
 .../trident/state/RedisClusterStateUpdater.java|  6 ++---
 .../storm/redis/trident/state/RedisMapState.java   |  4 +--
 .../storm/redis/trident/state/RedisState.java  |  8 +++---
 .../redis/trident/state/RedisStateQuerier.java |  2 +-
 .../redis/trident/state/RedisStateUpdater.java |  6 ++---
 28 files changed, 116 insertions(+), 114 deletions(-)

diff --git a/external/storm-redis/pom.xml b/external/storm-redis/pom.xml
index 5c90279..1938d46 100644
--- a/external/storm-redis/pom.xml
+++ b/external/storm-redis/pom.xml
@@ -86,7 +86,7 @@
 maven-checkstyle-plugin
 
 
-64
+0
 
 
 
diff --git 
a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
 
b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
index 06d59b6..fd49f91 100644
--- 
a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
+++ 
b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/AbstractRedisBolt.java
@@ -24,12 +24,12 @@ import redis.clients.jedis.JedisCommands;
 
 /**
  * AbstractRedisBolt class is for users to implement custom bolts which makes 
interaction with Redis.
- * 
- * Due to environment abstraction, AbstractRedisBolt provides JedisCommands 
which contains only single key operations.
- * 
- * Custom Bolts may want to follow this pattern:
- * 
- * 
+ *
+ * Due to environment abstraction, AbstractRedisBolt provides JedisCommands 
which contains only single key operations.
+ *
+ * Custom Bolts may want to follow this pattern:
+ *
+ * 
  * JedisCommands jedisCommands = null;
  * try {
  * jedisCommand = getInstance();
diff --git 
a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
 
b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
index 1b7ab20..a3f0ace 100644
--- 
a/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
+++ 
b/external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
@@ -25,16 +25,16 @@ import redis.clients.jedis.JedisCommands;
 /**
  * Basic bolt for querying from Redis and filters out if key/field doesn't 
exist.
  * If key/field exists on Redis, this bolt just forwards input tuple to 
default stream.
- * 
- * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
- * 
- * Note: For STRING it checks such key exists on the key space.
+ *
+ * Supported data types: STRING, HASH, SET, SORTED_SET, HYPER_LOG_LOG, GEO.
+ *
+ * Note: For STRING it checks such key exists on the key space.
  * For HASH and SORTED_SET and GEO, it checks such field exists on that data 
structure.
  * For SET and HYPER_LOG_LOG, it check such value exists on that data 
structure.
  * (Note that it still refers key

[storm] branch master updated: STORM-3438: webapp: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4b1d2aa  STORM-3438: webapp: fix all checkstyle warnings
 new 994a484  Merge pull request #3054 from krichter722/checkstyle-webapp
4b1d2aa is described below

commit 4b1d2aa21db2cf61d82c1e76a9de5c5cc935ab48
Author: Karl-Philipp Richter 
AuthorDate: Sun Jun 30 11:50:27 2019 +0200

STORM-3438: webapp: fix all checkstyle warnings
---
 storm-webapp/pom.xml   |  2 +-
 .../common/AuthorizationExceptionMapper.java   |  4 +-
 .../org/apache/storm/daemon/drpc/DRPCServer.java   |  1 +
 .../storm/daemon/drpc/webapp/DRPCApplication.java  |  1 +
 .../daemon/drpc/webapp/DRPCExceptionMapper.java|  1 +
 .../storm/daemon/drpc/webapp/DRPCResource.java |  1 +
 .../handler/LogviewerLogSearchHandler.java |  7 ++-
 .../java/org/apache/storm/daemon/ui/UIHelpers.java | 63 +++---
 .../java/org/apache/storm/daemon/ui/UIServer.java  |  5 +-
 .../daemon/ui/filters/AuthorizedUserFilter.java| 12 ++---
 10 files changed, 53 insertions(+), 44 deletions(-)

diff --git a/storm-webapp/pom.xml b/storm-webapp/pom.xml
index b0ada88..8d42589 100644
--- a/storm-webapp/pom.xml
+++ b/storm-webapp/pom.xml
@@ -309,7 +309,7 @@
 maven-checkstyle-plugin
 
 
-23
+0
 
 
 
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/common/AuthorizationExceptionMapper.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/common/AuthorizationExceptionMapper.java
index 251ada8..3dc4d2a 100644
--- 
a/storm-webapp/src/main/java/org/apache/storm/daemon/common/AuthorizationExceptionMapper.java
+++ 
b/storm-webapp/src/main/java/org/apache/storm/daemon/common/AuthorizationExceptionMapper.java
@@ -18,6 +18,8 @@
 
 package org.apache.storm.daemon.common;
 
+import static 
org.apache.storm.daemon.ui.exceptionmappers.ExceptionMapperUtils.getResponse;
+
 import java.util.HashMap;
 import java.util.Map;
 
@@ -30,8 +32,6 @@ import javax.ws.rs.ext.Provider;
 import org.apache.storm.generated.AuthorizationException;
 import org.json.simple.JSONValue;
 
-import static 
org.apache.storm.daemon.ui.exceptionmappers.ExceptionMapperUtils.getResponse;
-
 @Provider
 public class AuthorizationExceptionMapper implements 
ExceptionMapper {
 
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
index 9c24f06..7a8b5b3 100644
--- a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
+++ b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/DRPCServer.java
@@ -50,6 +50,7 @@ import org.glassfish.jersey.servlet.ServletContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class DRPCServer implements AutoCloseable {
 private static final Logger LOG = 
LoggerFactory.getLogger(DRPCServer.class);
 private final Meter meterShutdownCalls;
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
index f88bb00..26b2339 100644
--- 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
+++ 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCApplication.java
@@ -29,6 +29,7 @@ import org.apache.storm.daemon.drpc.DRPC;
 import org.apache.storm.metric.StormMetricsRegistry;
 
 @ApplicationPath("")
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class DRPCApplication extends Application {
 private static DRPC _drpc;
 private static StormMetricsRegistry metricsRegistry;
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCExceptionMapper.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCExceptionMapper.java
index 98876a9..85813c0 100644
--- 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCExceptionMapper.java
+++ 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCExceptionMapper.java
@@ -30,6 +30,7 @@ import org.apache.storm.generated.DRPCExecutionException;
 import org.json.simple.JSONValue;
 
 @Provider
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class DRPCExceptionMapper implements 
ExceptionMapper {
 
 @Override
diff --git 
a/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
 
b/storm-webapp/src/main/java/org/apache/storm/daemon/drpc/webapp/DRPCResource.java
index dba12cf..5056e09 100644
--- 
a/storm-webapp/src/main/java/org/apache/st

[storm] branch master updated: STORM-3433: pmml: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new d5434f5  STORM-3433: pmml: fix all checkstyle warnings
 new a4a1070  Merge pull request #3049 from krichter722/checkstyle-pmml
d5434f5 is described below

commit d5434f5e20effcea7be05c28416528470d90bc30
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 23:56:55 2019 +0200

STORM-3433: pmml: fix all checkstyle warnings
---
 external/storm-pmml/pom.xml|  2 +-
 .../org/apache/storm/pmml/PMMLPredictorBolt.java   | 21 ---
 .../org/apache/storm/pmml/model/ModelOutputs.java  | 12 ++--
 .../storm/pmml/model/jpmml/JpmmlModelOutputs.java  | 20 +++---
 .../org/apache/storm/pmml/runner/ModelRunner.java  |  7 ++-
 .../apache/storm/pmml/runner/PmmlModelRunner.java  |  5 +-
 .../storm/pmml/runner/jpmml/JPmmlModelRunner.java  | 20 +++---
 .../storm/pmml/runner/jpmml/JpmmlFactory.java  | 73 +-
 8 files changed, 89 insertions(+), 71 deletions(-)

diff --git a/external/storm-pmml/pom.xml b/external/storm-pmml/pom.xml
index 1c9278b..0a834ff 100644
--- a/external/storm-pmml/pom.xml
+++ b/external/storm-pmml/pom.xml
@@ -94,7 +94,7 @@
 
 
 
-67
+0
 
 
 
diff --git 
a/external/storm-pmml/src/main/java/org/apache/storm/pmml/PMMLPredictorBolt.java
 
b/external/storm-pmml/src/main/java/org/apache/storm/pmml/PMMLPredictorBolt.java
index e7825c0..0d9bc4c 100644
--- 
a/external/storm-pmml/src/main/java/org/apache/storm/pmml/PMMLPredictorBolt.java
+++ 
b/external/storm-pmml/src/main/java/org/apache/storm/pmml/PMMLPredictorBolt.java
@@ -18,6 +18,9 @@
 
 package org.apache.storm.pmml;
 
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.pmml.model.ModelOutputs;
 import org.apache.storm.pmml.runner.ModelRunner;
 import org.apache.storm.pmml.runner.ModelRunnerFactory;
@@ -32,9 +35,7 @@ import org.apache.storm.utils.TupleUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.List;
-import java.util.Map;
-
+@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
 public class PMMLPredictorBolt extends BaseTickTupleAwareRichBolt {
 protected static final Logger LOG = 
LoggerFactory.getLogger(PMMLPredictorBolt.class);
 
@@ -52,7 +53,7 @@ public class PMMLPredictorBolt extends 
BaseTickTupleAwareRichBolt {
  * Creates an instance of {@link PMMLPredictorBolt} that executes, for 
every tuple, the runner constructed with
  * the {@link ModelRunnerFactory} specified in the parameter
  * The {@link PMMLPredictorBolt} instantiated with this constructor 
declares the output fields as specified
- * by the {@link ModelOutputs} parameter
+ * by the {@link ModelOutputs} parameter.
  */
 public PMMLPredictorBolt(ModelRunnerFactory modelRunnerFactory, 
ModelOutputs modelOutputs) {
 this.outputs = modelOutputs;
@@ -95,11 +96,11 @@ public class PMMLPredictorBolt extends 
BaseTickTupleAwareRichBolt {
 
 @Override
 public String toString() {
-return "PMMLPredictorBolt{" +
-"outputFields=" + outputs +
-", runnerFactory=" + runnerFactory.getClass().getName() +
-", runner=" + runner +
-", collector=" + collector +
-"} ";
+return "PMMLPredictorBolt{"
++ "outputFields=" + outputs
++ ", runnerFactory=" + runnerFactory.getClass().getName()
++ ", runner=" + runner
++ ", collector=" + collector
++ "} ";
 }
 }
diff --git 
a/external/storm-pmml/src/main/java/org/apache/storm/pmml/model/ModelOutputs.java
 
b/external/storm-pmml/src/main/java/org/apache/storm/pmml/model/ModelOutputs.java
index 5f61045..8322a8a 100644
--- 
a/external/storm-pmml/src/main/java/org/apache/storm/pmml/model/ModelOutputs.java
+++ 
b/external/storm-pmml/src/main/java/org/apache/storm/pmml/model/ModelOutputs.java
@@ -18,25 +18,27 @@
 
 package org.apache.storm.pmml.model;
 
-import org.apache.storm.pmml.PMMLPredictorBolt;
-import org.apache.storm.tuple.Fields;
-
 import java.io.Serializable;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.storm.pmml.PMMLPredictorBolt;
+import org.apache.storm.tuple.Fields;
+
 /**
- * Represents the streams and output fields declared by the {@link 
PMMLPredictorBolt}
+ * Represents the streams and output fields declared by the {@link 
PMMLPredictorBolt}.
  */
 public interface ModelOutputs extends Serializable {
+
 /**
+ * Stream fields.
  * @return a map with the output fields declared for each stream by 

[storm] branch master updated: STORM-3432: solr-examples: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 871c20f  STORM-3432: solr-examples: fix all checkstyle warnings
 new 220338f  Merge pull request #3048 from 
krichter722/checkstyle-solr-examples
871c20f is described below

commit 871c20f524930cf9d3e9c03b08266d33aea04d65
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 23:41:31 2019 +0200

STORM-3432: solr-examples: fix all checkstyle warnings
---
 examples/storm-solr-examples/pom.xml   |  2 +-
 .../apache/storm/solr/spout/SolrFieldsSpout.java   | 13 +--
 .../org/apache/storm/solr/spout/SolrJsonSpout.java | 25 +++---
 .../storm/solr/topology/SolrFieldsTopology.java| 19 
 .../storm/solr/topology/SolrJsonTopology.java  |  6 +++---
 .../solr/trident/SolrFieldsTridentTopology.java|  6 +++---
 .../solr/trident/SolrJsonTridentTopology.java  |  6 +++---
 7 files changed, 40 insertions(+), 37 deletions(-)

diff --git a/examples/storm-solr-examples/pom.xml 
b/examples/storm-solr-examples/pom.xml
index ea89a03..81570d1 100644
--- a/examples/storm-solr-examples/pom.xml
+++ b/examples/storm-solr-examples/pom.xml
@@ -118,7 +118,7 @@
 maven-checkstyle-plugin
 
 
-47
+0
 
 
 
diff --git 
a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrFieldsSpout.java
 
b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrFieldsSpout.java
index 2959eed..70b0e08 100644
--- 
a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrFieldsSpout.java
+++ 
b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrFieldsSpout.java
@@ -18,18 +18,19 @@
 
 package org.apache.storm.solr.spout;
 
+import com.google.common.collect.Lists;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.storm.solr.util.TestUtil;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import com.google.common.collect.Lists;
-import org.apache.storm.solr.util.TestUtil;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
 
 public class SolrFieldsSpout extends BaseRichSpout {
 private SpoutOutputCollector collector;
diff --git 
a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrJsonSpout.java
 
b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrJsonSpout.java
index 4f52668..ed3fa76 100644
--- 
a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrJsonSpout.java
+++ 
b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/spout/SolrJsonSpout.java
@@ -18,20 +18,21 @@
 
 package org.apache.storm.solr.spout;
 
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
 import com.google.common.collect.Lists;
 import com.google.gson.Gson;
-import org.apache.storm.solr.util.TestUtil;
 
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import org.apache.storm.solr.util.TestUtil;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+
 public class SolrJsonSpout extends BaseRichSpout {
 private SpoutOutputCollector collector;
 private static final List listValues = Lists.newArrayList(
@@ -80,27 +81,27 @@ public class SolrJsonSpout extends BaseRichSpout {
 public static class JsonSchema {
 private String id;
 private String date;
-private String dc_title;
+private String dcTitle;
 
 private static final Gson gson = new Gson();
 
 public JsonSchema(String suffix) {
 this.id = "id" + suffix;
 this.date = TestUtil.getDate();
-this.dc_title = "dc_title" + suffix;
+this.dcTitle = "dcTitle" + suffix;
 }
 
-public JsonSchema(String id, String date, String dc_title) {
+public JsonSchema(String id, String date, String dcTitle) {
 this.id = id;
 this.date = date;
-this.dc_title = dc_ti

[storm] branch master updated: STORM-3431: rocketmq-examples: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 052e082  STORM-3431: rocketmq-examples: fix all checkstyle warnings
 new ead0bdd  Merge pull request #3047 from 
krichter722/checkstyle-rocketmq-examples
052e082 is described below

commit 052e082376a4640987993148d000e6b12922550d
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 23:22:20 2019 +0200

STORM-3431: rocketmq-examples: fix all checkstyle warnings
---
 examples/storm-rocketmq-examples/pom.xml | 2 +-
 .../java/org/apache/storm/rocketmq/topology/WordCountTopology.java   | 5 ++---
 .../java/org/apache/storm/rocketmq/trident/WordCountTrident.java | 2 +-
 3 files changed, 4 insertions(+), 5 deletions(-)

diff --git a/examples/storm-rocketmq-examples/pom.xml 
b/examples/storm-rocketmq-examples/pom.xml
index de57c51..31774a4 100644
--- a/examples/storm-rocketmq-examples/pom.xml
+++ b/examples/storm-rocketmq-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-23
+0
 
 
 
diff --git 
a/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/topology/WordCountTopology.java
 
b/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/topology/WordCountTopology.java
index 40d3eb4..3b40b51 100644
--- 
a/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/topology/WordCountTopology.java
+++ 
b/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/topology/WordCountTopology.java
@@ -41,7 +41,7 @@ public class WordCountTopology {
 private static final String CONSUMER_GROUP = "wordcount";
 private static final String CONSUMER_TOPIC = "source";
 
-public static StormTopology buildTopology(String nameserverAddr, String 
topic){
+public static StormTopology buildTopology(String nameserverAddr, String 
topic) {
 Properties properties = new Properties();
 properties.setProperty(SpoutConfig.NAME_SERVER_ADDR, nameserverAddr);
 properties.setProperty(SpoutConfig.CONSUMER_GROUP, CONSUMER_GROUP);
@@ -49,8 +49,6 @@ public class WordCountTopology {
 
 RocketMqSpout spout = new RocketMqSpout(properties);
 
-WordCounter bolt = new WordCounter();
-
 TupleToMessageMapper mapper = new 
FieldNameBasedTupleToMessageMapper("word", "count");
 TopicSelector selector = new DefaultTopicSelector(topic);
 
@@ -65,6 +63,7 @@ public class WordCountTopology {
 // wordSpout ==> countBolt ==> insertBolt
 TopologyBuilder builder = new TopologyBuilder();
 
+WordCounter bolt = new WordCounter();
 builder.setSpout(WORD_SPOUT, spout, 1);
 builder.setBolt(COUNT_BOLT, bolt, 1).fieldsGrouping(WORD_SPOUT, new 
Fields("str"));
 builder.setBolt(INSERT_BOLT, insertBolt, 
1).shuffleGrouping(COUNT_BOLT);
diff --git 
a/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/trident/WordCountTrident.java
 
b/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/trident/WordCountTrident.java
index 6b73d3f..dbdee93 100644
--- 
a/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/trident/WordCountTrident.java
+++ 
b/examples/storm-rocketmq-examples/src/main/java/org/apache/storm/rocketmq/trident/WordCountTrident.java
@@ -39,7 +39,7 @@ import org.apache.storm.tuple.Values;
 
 public class WordCountTrident {
 
-public static StormTopology buildTopology(String nameserverAddr, String 
topic){
+public static StormTopology buildTopology(String nameserverAddr, String 
topic) {
 Fields fields = new Fields("word", "count");
 FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
 new Values("storm", 1),



[storm] branch master updated: STORM-3430: flux-wrappers: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new d597e1d  STORM-3430: flux-wrappers: fix all checkstyle warnings
 new 1070293  Merge pull request #3046 from 
krichter722/checkstyle-flux-wrappers
d597e1d is described below

commit d597e1d076260491ee14a2a7975f32ceaf77ab5d
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 23:18:03 2019 +0200

STORM-3430: flux-wrappers: fix all checkstyle warnings
---
 flux/flux-wrappers/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/flux/flux-wrappers/pom.xml b/flux/flux-wrappers/pom.xml
index 6eb587b..d3131f7 100644
--- a/flux/flux-wrappers/pom.xml
+++ b/flux/flux-wrappers/pom.xml
@@ -55,7 +55,7 @@
 maven-checkstyle-plugin
 
 
-38
+0
 
 
 



[storm] branch master updated: STORM-3429: closure: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b203ae3  STORM-3429: closure: fix all checkstyle warnings
 new e80da42  Merge pull request #3045 from krichter722/checkstyle-closure
b203ae3 is described below

commit b203ae379b61117b0f795c3c408b81751cf8d3bc
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 23:06:03 2019 +0200

STORM-3429: closure: fix all checkstyle warnings
---
 storm-clojure/pom.xml  |   2 +-
 .../java/org/apache/storm/clojure/ClojureBolt.java |  77 +++---
 .../clojure/ClojureSerializationRegister.java  |   5 +-
 .../org/apache/storm/clojure/ClojureSpout.java | 101 ++-
 .../org/apache/storm/clojure/ClojureTuple.java | 112 -
 .../java/org/apache/storm/clojure/ClojureUtil.java |   1 +
 .../apache/storm/clojure/IndifferentAccessMap.java |  53 ++
 .../org/apache/storm/clojure/RichShellBolt.java|  14 +--
 .../org/apache/storm/clojure/RichShellSpout.java   |  14 +--
 9 files changed, 207 insertions(+), 172 deletions(-)

diff --git a/storm-clojure/pom.xml b/storm-clojure/pom.xml
index f886767..89d200b 100644
--- a/storm-clojure/pom.xml
+++ b/storm-clojure/pom.xml
@@ -110,7 +110,7 @@
 maven-checkstyle-plugin
 
 
-173
+0
 
 
 
diff --git 
a/storm-clojure/src/main/java/org/apache/storm/clojure/ClojureBolt.java 
b/storm-clojure/src/main/java/org/apache/storm/clojure/ClojureBolt.java
index af371a2..bc37400 100644
--- a/storm-clojure/src/main/java/org/apache/storm/clojure/ClojureBolt.java
+++ b/storm-clojure/src/main/java/org/apache/storm/clojure/ClojureBolt.java
@@ -15,8 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.clojure;
 
+import clojure.lang.IFn;
+import clojure.lang.Keyword;
+import clojure.lang.PersistentArrayMap;
+import clojure.lang.RT;
+import clojure.lang.Symbol;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -31,47 +38,43 @@ import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 
-import clojure.lang.IFn;
-import clojure.lang.Keyword;
-import clojure.lang.PersistentArrayMap;
-import clojure.lang.RT;
-import clojure.lang.Symbol;
-
 public class ClojureBolt implements IRichBolt, FinishedCallback {
-Map _fields;
-List _fnSpec;
-List _confSpec;
-List _params;
+Map fields;
+List fnSpec;
+List confSpec;
+List params;
 
-IBolt _bolt;
+IBolt bolt;
 
 public ClojureBolt(List fnSpec, List confSpec, List params, 
Map fields) {
-_fnSpec = fnSpec;
-_confSpec = confSpec;
-_params = params;
-_fields = fields;
+this.fnSpec = fnSpec;
+this.confSpec = confSpec;
+this.params = params;
+this.fields = fields;
 }
 
 @Override
 public void prepare(final Map topoConf, final 
TopologyContext context, final OutputCollector collector) {
-IFn hof = ClojureUtil.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1));
+IFn hof = ClojureUtil.loadClojureFn(fnSpec.get(0), fnSpec.get(1));
 try {
-IFn preparer = (IFn) hof.applyTo(RT.seq(_params));
-final Map collectorMap = new PersistentArrayMap( 
new Object[] {
+IFn preparer = (IFn) hof.applyTo(RT.seq(params));
+final Map collectorMap = new 
PersistentArrayMap(new Object[] {
 Keyword.intern(Symbol.create("output-collector")), 
collector,
 Keyword.intern(Symbol.create("context")), context});
-List args = new ArrayList() {{
-add(topoConf);
-add(context);
-add(collectorMap);
-}};
+List args = new ArrayList() {
+{
+add(topoConf);
+add(context);
+add(collectorMap);
+}
+};
 
-_bolt = (IBolt) preparer.applyTo(RT.seq(args));
+bolt = (IBolt) preparer.applyTo(RT.seq(args));
 //this is kind of unnecessary for clojure
 try {
-_bolt.prepare(topoConf, context, collector);
-} catch(AbstractMethodError ame) {
-
+bolt.prepare(topoConf, context, collector);
+} catch (AbstractMethodError ame) {
+//ignore
 }
 } catch (Exception e) {
 throw new RuntimeException(e);
@@ -80,38 +83,38 @@ public class ClojureBolt implements IRichBolt, 
FinishedCallback {
 
 @Override
 publi

[storm] branch master updated: STORM-3428: kafka-monitor: fix all checkstyle warnings

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b32d1d8  STORM-3428: kafka-monitor: fix all checkstyle warnings
 new b43793c  Merge pull request #3043 from 
krichter722/checkstyle-kafka-monitor
b32d1d8 is described below

commit b32d1d80e9b18527c189403035844c5b3449bb6a
Author: Karl-Philipp Richter 
AuthorDate: Thu Jun 27 21:54:59 2019 +0200

STORM-3428: kafka-monitor: fix all checkstyle warnings
---
 external/storm-kafka-monitor/pom.xml   |  2 +-
 .../storm/kafka/monitor/KafkaOffsetLagResult.java  | 48 ++
 .../storm/kafka/monitor/KafkaOffsetLagUtil.java| 38 ++---
 .../kafka/monitor/KafkaPartitionOffsetLag.java | 22 ++
 .../kafka/monitor/NewKafkaSpoutOffsetQuery.java| 18 
 5 files changed, 81 insertions(+), 47 deletions(-)

diff --git a/external/storm-kafka-monitor/pom.xml 
b/external/storm-kafka-monitor/pom.xml
index 99d2e62..cd18c76 100644
--- a/external/storm-kafka-monitor/pom.xml
+++ b/external/storm-kafka-monitor/pom.xml
@@ -75,7 +75,7 @@
 maven-checkstyle-plugin
 
 
-87
+0
 
 
 
diff --git 
a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagResult.java
 
b/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagResult.java
index 908141c..f6feff1 100644
--- 
a/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagResult.java
+++ 
b/external/storm-kafka-monitor/src/main/java/org/apache/storm/kafka/monitor/KafkaOffsetLagResult.java
@@ -21,7 +21,7 @@ package org.apache.storm.kafka.monitor;
 import org.json.simple.JSONAware;
 
 /**
- * Class representing the log head offsets, spout offsets and the lag for a 
topic
+ * Class representing the log head offsets, spout offsets and the lag for a 
topic.
  */
 public class KafkaOffsetLagResult implements JSONAware {
 private String topic;
@@ -60,26 +60,38 @@ public class KafkaOffsetLagResult implements JSONAware {
 
 @Override
 public String toString() {
-return "KafkaOffsetLagResult{" +
-   "topic='" + topic + '\'' +
-   ", partition=" + partition +
-   ", consumerCommittedOffset=" + consumerCommittedOffset +
-   ", logHeadOffset=" + logHeadOffset +
-   ", lag=" + lag +
-   '}';
+return "KafkaOffsetLagResult{"
++ "topic='" + topic + '\''
++ ", partition=" + partition
++ ", consumerCommittedOffset=" + consumerCommittedOffset
++ ", logHeadOffset=" + logHeadOffset
++ ", lag=" + lag
++ '}';
 }
 
 @Override
 public boolean equals(Object o) {
-if (this == o) return true;
-if (o == null || getClass() != o.getClass()) return false;
+if (this == o) {
+return true;
+}
+if (o == null || getClass() != o.getClass()) {
+return false;
+}
 
 KafkaOffsetLagResult that = (KafkaOffsetLagResult) o;
 
-if (partition != that.partition) return false;
-if (consumerCommittedOffset != that.consumerCommittedOffset) return 
false;
-if (logHeadOffset != that.logHeadOffset) return false;
-if (lag != that.lag) return false;
+if (partition != that.partition) {
+return false;
+}
+if (consumerCommittedOffset != that.consumerCommittedOffset) {
+return false;
+}
+if (logHeadOffset != that.logHeadOffset) {
+return false;
+}
+if (lag != that.lag) {
+return false;
+}
 return !(topic != null ? !topic.equals(that.topic) : that.topic != 
null);
 
 }
@@ -96,7 +108,11 @@ public class KafkaOffsetLagResult implements JSONAware {
 
 @Override
 public String toJSONString() {
-return "{\"topic\":\"" + topic + "\",\"partition\":" + partition + 
",\"consumerCommittedOffset\":" + consumerCommittedOffset + "," +
-   "\"logHeadOffset\":" + logHeadOffset + ",\"lag\":" + lag + "}";
+return "{\"topic\":\"" + topic
++ "\",\"partition\":" + partition
++ ",\"consumerCommittedOffset\":" + consumerCommittedOffset
++ ",\"logHeadOffset\":" + logHeadOffset
+ 

[storm] branch master updated: STORM-3420: Fix checkstyle violations in storm-mqtt

2019-07-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 8f9fedf  STORM-3420: Fix checkstyle violations in storm-mqtt
 new 91c8cde  Merge pull request #3037 from krichter722/checkstyle-mqtt
8f9fedf is described below

commit 8f9fedf37ba09f0d0d75e6d0232da10a8767833f
Author: Karl-Philipp Richter 
AuthorDate: Mon Jun 24 00:33:15 2019 +0200

STORM-3420: Fix checkstyle violations in storm-mqtt
---
 external/storm-mqtt/pom.xml|  2 +-
 .../org/apache/storm/mqtt/MqttTupleMapper.java |  4 +--
 .../org/apache/storm/mqtt/common/MqttOptions.java  | 32 ++
 .../apache/storm/mqtt/common/MqttPublisher.java|  2 +-
 .../org/apache/storm/mqtt/common/MqttUtils.java|  2 +-
 .../org/apache/storm/mqtt/common/SslUtils.java |  5 ++--
 .../org/apache/storm/mqtt/spout/MqttSpout.java | 11 
 7 files changed, 15 insertions(+), 43 deletions(-)

diff --git a/external/storm-mqtt/pom.xml b/external/storm-mqtt/pom.xml
index 35a237a..ad9ffcf 100644
--- a/external/storm-mqtt/pom.xml
+++ b/external/storm-mqtt/pom.xml
@@ -138,7 +138,7 @@
 maven-checkstyle-plugin
 
 
-39
+0
 
 
 
diff --git 
a/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/MqttTupleMapper.java 
b/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/MqttTupleMapper.java
index d3762b0..58473b7 100644
--- 
a/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/MqttTupleMapper.java
+++ 
b/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/MqttTupleMapper.java
@@ -12,8 +12,8 @@
 
 package org.apache.storm.mqtt;
 
-
 import java.io.Serializable;
+
 import org.apache.storm.tuple.ITuple;
 
 /**
@@ -22,7 +22,7 @@ import org.apache.storm.tuple.ITuple;
 public interface MqttTupleMapper extends Serializable {
 
 /**
- * Converts a Tuple to a MqttMessage
+ * Converts a Tuple to a MqttMessage.
  * @param tuple the incoming tuple
  * @return the message to publish
  */
diff --git 
a/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/common/MqttOptions.java
 
b/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/common/MqttOptions.java
index 1e9c925..90c9222 100644
--- 
a/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/common/MqttOptions.java
+++ 
b/external/storm-mqtt/src/main/java/org/apache/storm/mqtt/common/MqttOptions.java
@@ -16,7 +16,7 @@ import java.io.Serializable;
 import java.util.List;
 
 /**
- * MQTT Configuration Options
+ * MQTT Configuration Options.
  */
 public class MqttOptions implements Serializable {
 private String url = "tcp://localhost:1883";
@@ -44,10 +44,7 @@ public class MqttOptions implements Serializable {
 }
 
 /**
- * Sets the url for connecting to the MQTT broker.
- *
- * Default: `tcp://localhost:1883'
- * @param url
+ * Sets the url for connecting to the MQTT broker, e.g. {@code 
tcp://localhost:1883}.
  */
 public void setUrl(String url) {
 this.url = url;
@@ -59,8 +56,6 @@ public class MqttOptions implements Serializable {
 
 /**
  * A list of MQTT topics to subscribe to.
- *
- * @param topics
  */
 public void setTopics(List topics) {
 this.topics = topics;
@@ -73,8 +68,6 @@ public class MqttOptions implements Serializable {
 /**
  * Set to false if you want the MQTT server to persist topic subscriptions 
and ack positions across client sessions.
  * Defaults to false.
- *
- * @param cleanConnection
  */
 public void setCleanConnection(boolean cleanConnection) {
 this.cleanConnection = cleanConnection;
@@ -87,8 +80,6 @@ public class MqttOptions implements Serializable {
 /**
  * If set the server will publish the client's Will message to the 
specified topics if the client has an unexpected
  * disconnection.
- *
- * @param willTopic
  */
 public void setWillTopic(String willTopic) {
 this.willTopic = willTopic;
@@ -100,8 +91,6 @@ public class MqttOptions implements Serializable {
 
 /**
  * The Will message to send. Defaults to a zero length message.
- *
- * @param willPayload
  */
 public void setWillPayload(String willPayload) {
 this.willPayload = willPayload;
@@ -113,8 +102,6 @@ public class MqttOptions implements Serializable {
 
 /**
  * How long to wait in ms before the first reconnect attempt. Defaults to 
10.
- *
- * @param reconnectDelay
  */
 public void setReconnectDelay(long reconnectDelay) {
 this.reconnectDelay = reconnectDelay;
@@ -126,8 +113,6 @@ public class MqttOptions implements Serializable {
 
 /**
  * The maximum amount of time in ms to wait between

[storm] branch master updated: STORM-3419: Fix checkstyle violations in storm-opentsdb

2019-06-27 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new bf0a2e5  STORM-3419: Fix checkstyle violations in storm-opentsdb
 new b4f9eb1  Merge pull request #3038 from krichter722/checkstyle-opentsdb
bf0a2e5 is described below

commit bf0a2e51ccf36fec193f221ab40cd81218603ef2
Author: Karl-Philipp Richter 
AuthorDate: Mon Jun 24 00:19:33 2019 +0200

STORM-3419: Fix checkstyle violations in storm-opentsdb
---
 external/storm-opentsdb/pom.xml|  2 +-
 .../storm/opentsdb/OpenTsdbMetricDatapoint.java| 39 
 .../bolt/ITupleOpenTsdbDatapointMapper.java|  8 +--
 .../apache/storm/opentsdb/bolt/OpenTsdbBolt.java   | 25 
 .../bolt/TupleOpenTsdbDatapointMapper.java | 60 ---
 .../storm/opentsdb/client/ClientResponse.java  | 70 ++
 .../storm/opentsdb/client/OpenTsdbClient.java  | 20 ---
 .../storm/opentsdb/trident/OpenTsdbState.java  | 15 +++--
 .../opentsdb/trident/OpenTsdbStateFactory.java | 11 ++--
 .../opentsdb/trident/OpenTsdbStateUpdater.java |  5 +-
 10 files changed, 158 insertions(+), 97 deletions(-)

diff --git a/external/storm-opentsdb/pom.xml b/external/storm-opentsdb/pom.xml
index c2b311c..dd6eca0 100644
--- a/external/storm-opentsdb/pom.xml
+++ b/external/storm-opentsdb/pom.xml
@@ -106,7 +106,7 @@
 maven-checkstyle-plugin
 
 
-99
+0
 
 
 
diff --git 
a/external/storm-opentsdb/src/main/java/org/apache/storm/opentsdb/OpenTsdbMetricDatapoint.java
 
b/external/storm-opentsdb/src/main/java/org/apache/storm/opentsdb/OpenTsdbMetricDatapoint.java
index b04d817..0b47e10 100644
--- 
a/external/storm-opentsdb/src/main/java/org/apache/storm/opentsdb/OpenTsdbMetricDatapoint.java
+++ 
b/external/storm-opentsdb/src/main/java/org/apache/storm/opentsdb/OpenTsdbMetricDatapoint.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.storm.opentsdb;
 
 import java.io.Serializable;
@@ -57,6 +58,7 @@ public class OpenTsdbMetricDatapoint implements Serializable {
 }
 
 /**
+ * Retrieve the metric name of this datapoint.
  * @return metric name of this datapoint
  */
 public String getMetric() {
@@ -64,6 +66,7 @@ public class OpenTsdbMetricDatapoint implements Serializable {
 }
 
 /**
+ * Retrieve the map of tag/value pairs of this metric.
  * @return Map of tag/value pairs of this metric
  */
 public Map getTags() {
@@ -71,13 +74,15 @@ public class OpenTsdbMetricDatapoint implements 
Serializable {
 }
 
 /**
- * @return timestamp either in milliseconds or seconds at which this 
metric is occurred.
+ * Retrieve the timestamp at which this metric occured.
+ * @return timestamp either in milliseconds or seconds at which this 
metric occurred
  */
 public long getTimestamp() {
 return timestamp;
 }
 
 /**
+ * Retrieve the value of this metric datapoint.
  * @return value of this metric datapoint
  */
 public Object getValue() {
@@ -86,24 +91,34 @@ public class OpenTsdbMetricDatapoint implements 
Serializable {
 
 @Override
 public String toString() {
-return "OpenTsdbMetricDataPoint{" +
-"metric='" + metric + '\'' +
-", tags=" + tags +
-", timestamp=" + timestamp +
-", value=" + value +
-'}';
+return "OpenTsdbMetricDataPoint{"
++ "metric='" + metric + '\''
++ ", tags=" + tags
++ ", timestamp=" + timestamp
++ ", value=" + value
++ '}';
 }
 
 @Override
 public boolean equals(Object o) {
-if (this == o) return true;
-if (!(o instanceof OpenTsdbMetricDatapoint)) return false;
+if (this == o) {
+return true;
+}
+if (!(o instanceof OpenTsdbMetricDatapoint)) {
+return false;
+}
 
 OpenTsdbMetricDatapoint that = (OpenTsdbMetricDatapoint) o;
 
-if (timestamp != that.timestamp) return false;
-if (value != that.value) return false;
-if (!metric.equals(that.metric)) return false;
+if (timestamp != that.timestamp) {
+return false;
+}
+if (value != that.value) {
+return false;
+}
+if (!metric.equals(that.metric)) {
+return false;
+}
 return tags.equals(that.tags);
 
 }
diff --git 
a/external/storm-opentsdb/src/main/java/org/apache/storm/opentsd

[storm] branch master updated: STORM-3427: integration-test: fix all checkstyle warnings

2019-06-27 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new de44647  STORM-3427: integration-test: fix all checkstyle warnings
 new 9b23e8a  Merge pull request #3041 from 
krichter722/checkstyle-integration-test
de44647 is described below

commit de44647c76e3133ce4343b81eeda43db6fd2f7e9
Author: Karl-Philipp Richter 
AuthorDate: Tue Jun 25 22:06:51 2019 +0200

STORM-3427: integration-test: fix all checkstyle warnings
---
 integration-test/pom.xml   |   2 +-
 .../java/org/apache/storm/ExclamationTopology.java | 150 ++---
 .../java/org/apache/storm/debug/DebugHelper.java   |   6 +-
 .../apache/storm/st/topology/TestableTopology.java |   7 +-
 .../st/topology/window/IncrementingSpout.java  |   4 +-
 .../st/topology/window/SlidingTimeCorrectness.java |  10 +-
 .../topology/window/SlidingWindowCorrectness.java  |  12 +-
 .../topology/window/TimeDataIncrementingSpout.java |  52 +++
 .../topology/window/TumblingTimeCorrectness.java   |  12 +-
 .../topology/window/TumblingWindowCorrectness.java |  12 +-
 .../storm/st/topology/window/data/TimeData.java|  24 ++--
 .../org/apache/storm/st/utils/StringDecorator.java |   6 +-
 .../java/org/apache/storm/st/utils/TimeUtil.java   |   5 +-
 13 files changed, 163 insertions(+), 139 deletions(-)

diff --git a/integration-test/pom.xml b/integration-test/pom.xml
index 65f0453..e92fc97 100755
--- a/integration-test/pom.xml
+++ b/integration-test/pom.xml
@@ -175,7 +175,7 @@
 maven-checkstyle-plugin
 
 
-129
+0
 
 
 
diff --git 
a/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java 
b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
index fa44a98..d674e8c 100644
--- a/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
+++ b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
@@ -17,110 +17,110 @@
 
 package org.apache.storm;
 
-import com.google.common.collect.Lists;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import org.apache.storm.generated.StormTopology;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.st.topology.TestableTopology;
+import org.apache.storm.st.utils.TimeUtil;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseRichBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
 
-import java.util.Map;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.st.topology.TestableTopology;
-import org.apache.storm.st.utils.TimeUtil;
-import org.apache.storm.topology.base.BaseRichSpout;
-
 /**
  * This is a basic example of a Storm topology.
  */
 public class ExclamationTopology {
 
-  public static final String WORD = "word";
-  public static final String EXCLAIM_1 = "exclaim1";
-  public static final String EXCLAIM_2 = "exclaim2";
-  public static final int SPOUT_EXECUTORS = 10;
-  public static final int EXCLAIM_2_EXECUTORS = 2;
+public static final String WORD = "word";
+public static final String EXCLAIM_1 = "exclaim1";
+public static final String EXCLAIM_2 = "exclaim2";
+public static final int SPOUT_EXECUTORS = 10;
+public static final int EXCLAIM_2_EXECUTORS = 2;
 
-  public static class ExclamationBolt extends BaseRichBolt {
+public static class ExclamationBolt extends BaseRichBolt {
 
-OutputCollector _collector;
+OutputCollector collector;
 
-@Override
-public void prepare(Map conf, TopologyContext context, 
OutputCollector collector) {
-  _collector = collector;
-}
+@Override
+public void prepare(Map conf, TopologyContext context, 
OutputCollector collector) {
+this.collector = collector;
+}
 
-@Override
-public void execute(Tuple tuple) {
-  _collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
-  _collector.ack(tuple);
-}
+@Override
+public void execute(Tuple tuple) {
+collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
+collector.ack(tuple);
+}
 
-@Override
-public void declareOutputFields(OutputFieldsDeclarer declarer) {
-  declarer.declare(new Fields("word"));

[storm] branch master updated: STORM-3424: storm-submit-tools: fix all checkstyle warnings

2019-06-27 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 443d99a  STORM-3424: storm-submit-tools: fix all checkstyle warnings
 new 08d2b9e  Merge pull request #3039 from 
krichter722/checkstyle-submit-tools
443d99a is described below

commit 443d99a8da9e346763887d73914b42ceeabab3d9
Author: Karl-Philipp Richter 
AuthorDate: Tue Jun 25 21:21:49 2019 +0200

STORM-3424: storm-submit-tools: fix all checkstyle warnings
---
 storm-submit-tools/pom.xml |  2 +-
 .../submit/command/DependencyResolverMain.java | 30 +-
 .../org/apache/storm/submit/dependency/Booter.java |  4 +--
 .../submit/dependency/RepositorySystemFactory.java |  4 +--
 4 files changed, 29 insertions(+), 11 deletions(-)

diff --git a/storm-submit-tools/pom.xml b/storm-submit-tools/pom.xml
index 2036d81..3ff57df 100644
--- a/storm-submit-tools/pom.xml
+++ b/storm-submit-tools/pom.xml
@@ -115,7 +115,7 @@
 maven-checkstyle-plugin
 
 
-38
+0
 
 
 
diff --git 
a/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
 
b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
index 908d238..777bf74 100644
--- 
a/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
+++ 
b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
@@ -223,12 +223,30 @@ public class DependencyResolverMain {
 
 private static Options buildOptions() {
 Options options = new Options();
-options.addOption(null, OPTION_ARTIFACTS_LONG, true, "REQUIRED string 
representation of artifacts");
-options.addOption(null, OPTION_ARTIFACT_REPOSITORIES_LONG, true, 
"OPTIONAL string representation of artifact repositories");
-options.addOption(null, OPTION_MAVEN_LOCAL_REPOSITORY_DIRECTORY_LONG, 
true, "OPTIONAL string representation of local maven repository directory 
path");
-options.addOption(null, OPTION_PROXY_URL_LONG, true, "OPTIONAL URL 
representation of proxy server");
-options.addOption(null, OPTION_PROXY_USERNAME_LONG, true, "OPTIONAL 
Username of proxy server (basic auth)");
-options.addOption(null, OPTION_PROXY_PASSWORD_LONG, true, "OPTIONAL 
Password of proxy server (basic auth)");
+options.addOption(null,
+OPTION_ARTIFACTS_LONG,
+true,
+"REQUIRED string representation of artifacts");
+options.addOption(null,
+OPTION_ARTIFACT_REPOSITORIES_LONG,
+true,
+"OPTIONAL string representation of artifact repositories");
+options.addOption(null,
+OPTION_MAVEN_LOCAL_REPOSITORY_DIRECTORY_LONG,
+true,
+"OPTIONAL string representation of local maven repository 
directory path");
+options.addOption(null,
+OPTION_PROXY_URL_LONG,
+true,
+"OPTIONAL URL representation of proxy server");
+options.addOption(null,
+OPTION_PROXY_USERNAME_LONG,
+true,
+"OPTIONAL Username of proxy server (basic auth)");
+options.addOption(null,
+OPTION_PROXY_PASSWORD_LONG,
+true,
+"OPTIONAL Password of proxy server (basic auth)");
 return options;
 }
 }
diff --git 
a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
 
b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
index 5b60faf..87b5e76 100644
--- 
a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
+++ 
b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
@@ -18,6 +18,8 @@
 
 package org.apache.storm.submit.dependency;
 
+import java.io.File;
+
 import org.apache.maven.repository.internal.MavenRepositorySystemUtils;
 import org.eclipse.aether.DefaultRepositorySystemSession;
 import org.eclipse.aether.RepositorySystem;
@@ -25,8 +27,6 @@ import org.eclipse.aether.RepositorySystemSession;
 import org.eclipse.aether.repository.LocalRepository;
 import org.eclipse.aether.repository.RemoteRepository;
 
-import java.io.File;
-
 /**
  * Manage mvn repository.
  */
diff --git 
a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
 
b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
index 2f358cd..0783baf 100644
--- 
a/storm-submi

[storm] branch master updated: STORM-3421: Fix checkstyle violations in storm-kinesis

2019-06-27 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 34750d5  STORM-3421: Fix checkstyle violations in storm-kinesis
 new f617c6e  Merge pull request #3036 from krichter722/checkstyle-kinesis
34750d5 is described below

commit 34750d557b82cbdfdef379ab9b7162142a06b169
Author: Karl-Philipp Richter 
AuthorDate: Mon Jun 24 01:15:03 2019 +0200

STORM-3421: Fix checkstyle violations in storm-kinesis
---
 external/storm-kinesis/pom.xml |   2 +-
 .../kinesis/spout/CredentialsProviderChain.java|   5 +-
 .../kinesis/spout/ExponentialBackoffRetrier.java   |  33 +++--
 .../kinesis/spout/FailedMessageRetryHandler.java   |  22 +--
 .../apache/storm/kinesis/spout/KinesisConfig.java  |  59 
 .../storm/kinesis/spout/KinesisConnection.java |  43 --
 .../storm/kinesis/spout/KinesisConnectionInfo.java |  26 ++--
 .../storm/kinesis/spout/KinesisMessageId.java  |  40 +++---
 .../storm/kinesis/spout/KinesisRecordsManager.java | 157 +
 .../apache/storm/kinesis/spout/KinesisSpout.java   |   8 +-
 .../storm/kinesis/spout/RecordToTupleMapper.java   |  12 +-
 .../spout/{ZKConnection.java => ZkConnection.java} |  26 ++--
 .../org/apache/storm/kinesis/spout/ZkInfo.java |  34 +++--
 13 files changed, 279 insertions(+), 188 deletions(-)

diff --git a/external/storm-kinesis/pom.xml b/external/storm-kinesis/pom.xml
index 6f04950..39c89f1 100644
--- a/external/storm-kinesis/pom.xml
+++ b/external/storm-kinesis/pom.xml
@@ -62,7 +62,7 @@
 maven-checkstyle-plugin
 
 
-185
+0
 
 
 
diff --git 
a/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/CredentialsProviderChain.java
 
b/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/CredentialsProviderChain.java
index 4287ae0..5820851 100644
--- 
a/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/CredentialsProviderChain.java
+++ 
b/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/CredentialsProviderChain.java
@@ -26,10 +26,11 @@ import 
com.amazonaws.auth.SystemPropertiesCredentialsProvider;
 import com.amazonaws.auth.profile.ProfileCredentialsProvider;
 
 /**
- * Class representing chain of mechanisms that will be used in order to 
connect to kinesis
+ * Class representing chain of mechanisms that will be used in order to 
connect to kinesis.
  */
 public class CredentialsProviderChain extends AWSCredentialsProviderChain {
-public CredentialsProviderChain () {
+
+public CredentialsProviderChain() {
 super(new EnvironmentVariableCredentialsProvider(),
 new SystemPropertiesCredentialsProvider(),
 new ClasspathPropertiesFileCredentialsProvider(),
diff --git 
a/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/ExponentialBackoffRetrier.java
 
b/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/ExponentialBackoffRetrier.java
index 88e8d70..2920913 100644
--- 
a/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/ExponentialBackoffRetrier.java
+++ 
b/external/storm-kinesis/src/main/java/org/apache/storm/kinesis/spout/ExponentialBackoffRetrier.java
@@ -18,9 +18,6 @@
 
 package org.apache.storm.kinesis.spout;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.Serializable;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -28,6 +25,9 @@ import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public class ExponentialBackoffRetrier implements FailedMessageRetryHandler, 
Serializable {
 private static final Logger LOG = 
LoggerFactory.getLogger(ExponentialBackoffRetrier.class);
 // Wait interfal for retrying after first failure
@@ -44,29 +44,29 @@ public class ExponentialBackoffRetrier implements 
FailedMessageRetryHandler, Ser
 private SortedSet retryMessageSet = new TreeSet<>(new 
RetryTimeComparator());
 
 /**
- * no args constructor that uses defaults of 100 ms for first retry, max 
retries of Long.MAX_VALUE and an exponential backoff of Math.pow(2,i-1) secs for
- * retry i where i = 2,3,
+ * No args constructor that uses defaults of 100 ms for first retry, max 
retries of Long.MAX_VALUE and an
+ * exponential backoff of {@code Math.pow(2,i-1)} secs for retry {@code i} 
where {@code i = 2,3,...}.
  */
-public ExponentialBackoffRetrier () {
+public ExponentialBackoffRetrier() {
 this(100L, 2L, Long.MAX_VALUE);
 }
 
 /**
- *
+ * Creates a new exponential backoff retrier.
  * @param initialDelayMillis delay in milliseconds for first retry
 

[storm] branch master updated: STORM-3417: Fix checkstyle violations in sql-core

2019-06-27 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new eb6a963  STORM-3417: Fix checkstyle violations in sql-core
 new 4822ecd  Merge pull request #3035 from krichter722/checkstyle-sql-core
eb6a963 is described below

commit eb6a963d05da12e5b604a15a5633adde95a27748
Author: Karl-Philipp Richter 
AuthorDate: Sat Jun 15 20:32:25 2019 +0200

STORM-3417: Fix checkstyle violations in sql-core
---
 sql/storm-sql-core/pom.xml |  2 +-
 .../src/jvm/org/apache/storm/sql/StormSql.java |  4 +-
 .../src/jvm/org/apache/storm/sql/StormSqlImpl.java |  5 +-
 .../jvm/org/apache/storm/sql/StormSqlRunner.java   |  4 +-
 .../apache/storm/sql/compiler/CompilerUtil.java| 15 ++--
 .../sql/compiler/RexNodeToJavaCodeCompiler.java| 85 +++---
 .../storm/sql/javac/CompilingClassLoader.java  |  5 +-
 .../apache/storm/sql/planner/StormRelUtils.java|  8 +-
 .../sql/planner/streams/StreamsStormRuleSets.java  |  3 +-
 .../sql/planner/streams/rel/StreamsCalcRel.java| 10 +--
 .../streams/rel/StreamsStreamInsertRel.java|  5 +-
 .../planner/streams/rel/StreamsStreamScanRel.java  |  3 +-
 12 files changed, 76 insertions(+), 73 deletions(-)

diff --git a/sql/storm-sql-core/pom.xml b/sql/storm-sql-core/pom.xml
index b2b3777..73636f9 100644
--- a/sql/storm-sql-core/pom.xml
+++ b/sql/storm-sql-core/pom.xml
@@ -207,7 +207,7 @@
 maven-checkstyle-plugin
 
 
-59
+0
 
 
 
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java 
b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
index e57ffd8..e4715f8 100644
--- a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
@@ -19,8 +19,8 @@ import org.apache.storm.generated.SubmitOptions;
 /**
  * The StormSql class provides standalone, interactive interfaces to execute
  * SQL statements over streaming data.
- * 
- * The StormSql class is stateless. The user needs to submit the data
+ *
+ * The StormSql class is stateless. The user needs to submit the data
  * definition language (DDL) statements and the query statements in the same
  * batch.
  */
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java 
b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
index fff801d..2f30063 100644
--- a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
@@ -78,14 +78,13 @@ class StormSqlImpl extends StormSql {
 @Override
 public void explain(Iterable statements) throws Exception {
 for (String sql : statements) {
-StormParser parser = new StormParser(sql);
-SqlNode node = parser.impl().parseSqlStmtEof();
-
 
System.out.println("===");
 System.out.println("query>");
 System.out.println(sql);
 
System.out.println("---");
 
+StormParser parser = new StormParser(sql);
+SqlNode node = parser.impl().parseSqlStmtEof();
 if (node instanceof SqlCreateTable) {
 sqlContext.interpretCreateTable((SqlCreateTable) node);
 System.out.println("No plan presented on DDL");
diff --git 
a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java 
b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
index 3ff5394..edcc46d 100644
--- a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
@@ -56,8 +56,8 @@ public class StormSqlRunner {
 SubmitOptions submitOptions = new 
SubmitOptions(TopologyInitialStatus.ACTIVE);
 sql.submit(topoName, stmts, conf, submitOptions, null, null);
 } else {
-printUsageAndExit(options, "Either " + 
OPTION_SQL_TOPOLOGY_NAME_LONG + " or " + OPTION_SQL_EXPLAIN_LONG +
-   " must be presented");
+printUsageAndExit(options, "Either " + 
OPTION_SQL_TOPOLOGY_NAME_LONG
++ " or " + OPTION_SQL_EXPLAIN_LONG + " must be presented");
 }
 }
 
diff --git 
a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java 
b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
index 3cb0f96..a91199c 100644
--- a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUti

[storm] branch master updated: STORM-3408 - update rocks version (#3024)

2019-06-25 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new fb76dd1  STORM-3408 - update rocks version (#3024)
fb76dd1 is described below

commit fb76dd1c7dc39c4979f9cc921cf4a5930dfaa760
Author: Simon Cooper 
AuthorDate: Tue Jun 25 18:39:43 2019 +0100

STORM-3408 - update rocks version (#3024)

* STORM-3408 - update rocks version to 5.18.3
---
 pom.xml|   2 +-
 .../metricstore/rocksdb/RocksDbMetricsWriter.java  |  20 +-
 .../storm/metricstore/rocksdb/RocksDbStore.java| 219 +++--
 3 files changed, 135 insertions(+), 106 deletions(-)

diff --git a/pom.xml b/pom.xml
index 989fedb..5603831 100644
--- a/pom.xml
+++ b/pom.xml
@@ -329,7 +329,7 @@
 0.9.12
 2.3.5
 2.3.0
-5.8.6
+5.18.3
 
 
 provided
diff --git 
a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
 
b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
index 5ae8a0e..fae3865 100644
--- 
a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
+++ 
b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbMetricsWriter.java
@@ -182,7 +182,12 @@ public class RocksDbMetricsWriter implements Runnable, 
AutoCloseable {
 }
 
 // attempt to find the string in the database
-stringMetadata = store.rocksDbGetStringMetadata(type, s);
+try {
+stringMetadata = store.rocksDbGetStringMetadata(type, s);
+}
+catch (RocksDBException e) {
+throw new MetricException("Error reading metrics data", e);
+}
 if (stringMetadata != null) {
 // update to the latest timestamp and add to the string cache
 stringMetadata.update(metricTimestamp, type);
@@ -234,10 +239,15 @@ public class RocksDbMetricsWriter implements Runnable, 
AutoCloseable {
 // now scan all metadata and remove any matching string Ids from 
this list
 RocksDbKey firstPrefix = 
RocksDbKey.getPrefix(KeyType.METADATA_STRING_START);
 RocksDbKey lastPrefix = 
RocksDbKey.getPrefix(KeyType.METADATA_STRING_END);
-store.scanRange(firstPrefix, lastPrefix, (key, value) -> {
-unusedIds.remove(key.getMetadataStringId());
-return true; // process all metadata
-});
+try {
+store.scanRange(firstPrefix, lastPrefix, (key, value) -> {
+unusedIds.remove(key.getMetadataStringId());
+return true; // process all metadata
+});
+}
+catch (RocksDBException e) {
+throw new MetricException("Error reading metrics data", e);
+}
 }
 }
 
diff --git 
a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
 
b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
index 6b7617e..ba3f08b 100644
--- 
a/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
+++ 
b/storm-server/src/main/java/org/apache/storm/metricstore/rocksdb/RocksDbStore.java
@@ -274,7 +274,13 @@ public class RocksDbStore implements MetricStore, 
AutoCloseable {
 }
 
 // attempt to find the string in the database
-stringMetadata = rocksDbGetStringMetadata(type, s);
+try {
+stringMetadata = rocksDbGetStringMetadata(type, s);
+}
+catch (RocksDBException e) {
+throw new MetricException("Error reading metric data", e);
+}
+
 if (stringMetadata != null) {
 id = stringMetadata.getStringId();
 
@@ -290,7 +296,7 @@ public class RocksDbStore implements MetricStore, 
AutoCloseable {
 }
 
 // scans the database to look for a metadata string and returns the 
metadata info
-StringMetadata rocksDbGetStringMetadata(KeyType type, String s) {
+StringMetadata rocksDbGetStringMetadata(KeyType type, String s) throws 
RocksDBException {
 RocksDbKey firstKey = RocksDbKey.getInitialKey(type);
 RocksDbKey lastKey = RocksDbKey.getLastKey(type);
 final AtomicReference reference = new 
AtomicReference<>();
@@ -306,20 +312,21 @@ public class RocksDbStore implements MetricStore, 
AutoCloseable {
 }
 
 // scans from key start to the key before end, calling back until callback 
indicates not to process further
-void scanRange(RocksDbKey start, RocksDbKey end, RocksDbScanCallback fn) {
+void scanRange(RocksDbKey start, RocksDbKey end, RocksDbScanCallback fn) 
throws RocksDBException {
 try (ReadOptions ro = new ReadOptions()

[storm] branch master updated: STORM-3414: Remove JavadocMethod checkstyle rule

2019-06-24 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new ad3c642  STORM-3414: Remove JavadocMethod checkstyle rule
 new 70353fc  Merge pull request #3027 from 
krichter722/remove-missing-javadoc
ad3c642 is described below

commit ad3c64211ff56456056b23fae95db25bdece194e
Author: Karl-Philipp Richter 
AuthorDate: Sat Jun 15 20:35:16 2019 +0200

STORM-3414: Remove JavadocMethod checkstyle rule

Remove JavadocMethod checkstyle rule because it requires tons of Javadocs 
to be invented on files which have not been touched for years and whose Javadoc 
can be covered with clean code.
---
 storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml | 9 -
 1 file changed, 9 deletions(-)

diff --git a/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml 
b/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
index e7678ce..d1497c8 100644
--- a/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
+++ b/storm-checkstyle/src/main/resources/storm/storm_checkstyle.xml
@@ -244,15 +244,6 @@
 
 
 
-
-
-
-
-
-
-
-
-
 
 
 

[storm] branch master updated: STORM-3416: Fix checkstyle violations in storm-jdbc-example

2019-06-23 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new fd34fd6  STORM-3416: Fix checkstyle violations in storm-jdbc-example
 new b19f28b  Merge pull request #3026 from 
krichter722/checkstyle-jdbc-examples
fd34fd6 is described below

commit fd34fd69200d207f6e8cbd8c4404003335e0c2c2
Author: Karl-Philipp Richter 
AuthorDate: Fri Jun 14 10:47:51 2019 +0200

STORM-3416: Fix checkstyle violations in storm-jdbc-example
---
 docs/storm-jdbc.md | 12 +-
 examples/storm-jdbc-examples/pom.xml   |  2 +-
 .../org/apache/storm/jdbc/spout/UserSpout.java | 10 +---
 .../storm/jdbc/topology/AbstractUserTopology.java  | 22 +-
 ...eTopology.java => UserPersistenceTopology.java} | 26 -
 ...gy.java => UserPersistenceTridentTopology.java} | 27 ++
 external/storm-jdbc/README.md  | 12 +-
 7 files changed, 69 insertions(+), 42 deletions(-)

diff --git a/docs/storm-jdbc.md b/docs/storm-jdbc.md
index 66bedde..0cb2353 100644
--- a/docs/storm-jdbc.md
+++ b/docs/storm-jdbc.md
@@ -74,11 +74,11 @@ ConnectionProvider connectionProvider = new 
HikariCPConnectionProvider(hikariCon
 String tableName = "user_details";
 JdbcMapper simpleJdbcMapper = new SimpleJdbcMapper(tableName, 
connectionProvider);
 
-JdbcInsertBolt userPersistanceBolt = new JdbcInsertBolt(connectionProvider, 
simpleJdbcMapper)
+JdbcInsertBolt userPersistenceBolt = new JdbcInsertBolt(connectionProvider, 
simpleJdbcMapper)
 .withTableName("user")
 .withQueryTimeoutSecs(30);
 Or
-JdbcInsertBolt userPersistanceBolt = new JdbcInsertBolt(connectionProvider, 
simpleJdbcMapper)
+JdbcInsertBolt userPersistenceBolt = new JdbcInsertBolt(connectionProvider, 
simpleJdbcMapper)
 .withInsertQuery("insert into user values 
(?,?)")
 .withQueryTimeoutSecs(30); 
   
 ```
@@ -241,8 +241,8 @@ insert into user_department values (4, 4);
 select dept_name from department, user_department where department.dept_id = 
user_department.dept_id and user_department.user_id = ?;
 ```
 ### Execution
-Run the `org.apache.storm.jdbc.topology.UserPersistanceTopology` class using 
storm jar command. The class expects 5 args
-storm jar org.apache.storm.jdbc.topology.UserPersistanceTopology 
[topology name]
+Run the `org.apache.storm.jdbc.topology.UserPersistenceTopology` class using 
storm jar command. The class expects 5 args
+storm jar org.apache.storm.jdbc.topology.UserPersistenceTopology 
[topology name]
 
 To make it work with Mysql, you can add the following to the pom.xml
 
@@ -276,7 +276,7 @@ You can generate a single jar with dependencies using mvn 
assembly plugin. To us
 Mysql Example:
 
 ```
-storm jar 
~/repo/incubator-storm/external/storm-jdbc/target/storm-jdbc-0.10.0-SNAPSHOT-jar-with-dependencies.jar
 org.apache.storm.jdbc.topology.UserPersistanceTopology  
com.mysql.jdbc.jdbc2.optional.MysqlDataSource jdbc:mysql://localhost/test root 
password UserPersistenceTopology
+storm jar 
~/repo/incubator-storm/external/storm-jdbc/target/storm-jdbc-0.10.0-SNAPSHOT-jar-with-dependencies.jar
 org.apache.storm.jdbc.topology.UserPersistenceTopology  
com.mysql.jdbc.jdbc2.optional.MysqlDataSource jdbc:mysql://localhost/test root 
password UserPersistenceTopology
 ```
 
 You can execute a select query against the user table which should show newly 
inserted rows:
@@ -285,4 +285,4 @@ You can execute a select query against the user table which 
should show newly in
 select * from user;
 ```
 
-For trident you can view 
`org.apache.storm.jdbc.topology.UserPersistanceTridentTopology`.
+For trident you can view 
`org.apache.storm.jdbc.topology.UserPersistenceTridentTopology`.
diff --git a/examples/storm-jdbc-examples/pom.xml 
b/examples/storm-jdbc-examples/pom.xml
index e3278ee..c334379 100644
--- a/examples/storm-jdbc-examples/pom.xml
+++ b/examples/storm-jdbc-examples/pom.xml
@@ -87,7 +87,7 @@
 maven-checkstyle-plugin
 
 
-36
+0
 
 
 
diff --git 
a/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/spout/UserSpout.java
 
b/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/spout/UserSpout.java
index 06f107c..fd323c8 100644
--- 
a/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/spout/UserSpout.java
+++ 
b/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/spout/UserSpout.java
@@ -15,17 +15,21 @@
  * See the License for the spe

[storm] branch master updated: STORM-3415: Fix checkstyle violations in storm-jms-example

2019-06-23 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 7411b1d  STORM-3415: Fix checkstyle violations in storm-jms-example
 new 7a07b5e  Merge pull request #3025 from 
krichter722/checkstyle-jms-examples
7411b1d is described below

commit 7411b1d45d5f343bf2a1cd43e552a02319a3a711
Author: Karl-Philipp Richter 
AuthorDate: Fri Jun 14 10:47:51 2019 +0200

STORM-3415: Fix checkstyle violations in storm-jms-example
---
 examples/storm-jms-examples/pom.xml|  2 +-
 .../storm/jms/example/ExampleJmsTopology.java  | 20 +--
 .../org/apache/storm/jms/example/GenericBolt.java  | 15 ++---
 .../storm/jms/example/JsonTupleProducer.java   | 23 
 .../storm/jms/example/SpringJmsProvider.java   | 65 +++---
 5 files changed, 66 insertions(+), 59 deletions(-)

diff --git a/examples/storm-jms-examples/pom.xml 
b/examples/storm-jms-examples/pom.xml
index 1e0511b..56596f7 100644
--- a/examples/storm-jms-examples/pom.xml
+++ b/examples/storm-jms-examples/pom.xml
@@ -105,7 +105,7 @@
 maven-checkstyle-plugin
 
 
-78
+0
 
 
 
diff --git 
a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
 
b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
index 096a16e..dcf9d48 100644
--- 
a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
+++ 
b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.jms.example;
 
 import javax.jms.JMSException;
@@ -33,6 +34,9 @@ import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.ITuple;
 
+/**
+ * An example JMS topology.
+ */
 public class ExampleJmsTopology {
 public static final String JMS_QUEUE_SPOUT = "JMS_QUEUE_SPOUT";
 public static final String INTERMEDIATE_BOLT = "INTERMEDIATE_BOLT";
@@ -41,6 +45,11 @@ public class ExampleJmsTopology {
 public static final String JMS_TOPIC_SPOUT = "JMS_TOPIC_SPOUT";
 public static final String ANOTHER_BOLT = "ANOTHER_BOLT";
 
+/**
+ * The main method.
+ * @param args takes the topology name as first argument
+ * @throws Exception any expection occuring durch cluster setup or 
operation
+ */
 @SuppressWarnings("serial")
 public static void main(String[] args) throws Exception {
 
@@ -49,11 +58,6 @@ public class ExampleJmsTopology {
 "jms-activemq.xml", "jmsConnectionFactory",
 "notificationQueue");
 
-// JMS Topic provider
-JmsProvider jmsTopicProvider = new SpringJmsProvider(
-"jms-activemq.xml", "jmsConnectionFactory",
-"notificationTopic");
-
 // JMS Producer
 JmsTupleProducer producer = new JsonTupleProducer();
 
@@ -67,7 +71,6 @@ public class ExampleJmsTopology {
 
 // spout with 5 parallel instances
 builder.setSpout(JMS_QUEUE_SPOUT, queueSpout, 5);
-
 // intermediate bolt, subscribes to jms spout, anchors on tuples, and 
auto-acks
 builder.setBolt(INTERMEDIATE_BOLT,
 new GenericBolt("INTERMEDIATE_BOLT", true, true, new 
Fields("json")), 3).shuffleGrouping(
@@ -78,6 +81,11 @@ public class ExampleJmsTopology {
 builder.setBolt(FINAL_BOLT, new GenericBolt("FINAL_BOLT", true, true), 
3).shuffleGrouping(
 INTERMEDIATE_BOLT);
 
+// JMS Topic provider
+JmsProvider jmsTopicProvider = new SpringJmsProvider(
+"jms-activemq.xml", "jmsConnectionFactory",
+"notificationTopic");
+
 // bolt that subscribes to the intermediate bolt, and publishes to a 
JMS Topic
 JmsBolt jmsBolt = new JmsBolt();
 jmsBolt.setJmsProvider(jmsTopicProvider);
diff --git 
a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/GenericBolt.java
 
b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/GenericBolt.java
index 341a4dc..fe90440 100644
--- 
a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/GenericBolt.java
+++ 
b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/GenericBolt.java
@@ -15,27 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under th

[storm] branch master updated: STORM-3409: Build shaded deps with dependencies, instead of as a loose module, never upload Storm artifacts to Travis build cache to ensure a clean slate for each build.

2019-06-20 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 43dee13  STORM-3409: Build shaded deps with dependencies, instead of 
as a loose module, never upload Storm artifacts to Travis build cache to ensure 
a clean slate for each build. Remove license plugin configuration until we can 
upgrade to 1.21, as it breaks clean builds.
 new dfae3c7  Merge pull request #3031 from srdo/STORM-3409
43dee13 is described below

commit 43dee13cd3ab3af2a3e89290c3dc0a740fe19192
Author: Stig Rohde Døssing 
AuthorDate: Sat Jun 15 20:33:41 2019 +0200

STORM-3409: Build shaded deps with dependencies, instead of as a loose 
module, never upload Storm artifacts to Travis build cache to ensure a clean 
slate for each build. Remove license plugin configuration until we can upgrade 
to 1.21, as it breaks clean builds.
---
 .travis.yml|   4 ++
 dev-tools/travis/travis-install.sh |   7 +-
 dev-tools/travis/travis-script.sh  |   6 +-
 pom.xml| 129 +
 storm-client/pom.xml   |   4 --
 5 files changed, 11 insertions(+), 139 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 429f3d5..1a79f48 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -23,6 +23,10 @@ env:
 dist: trusty
 sudo: required
 
+before_cache:
+# Never upload Storm artifacts to the build cache, ensuring a clean slate 
every time
+  - rm -rf $HOME/.m2/repository/org/apache/storm
+
 language: java
 jdk:
   - oraclejdk8
diff --git a/dev-tools/travis/travis-install.sh 
b/dev-tools/travis/travis-install.sh
index a5e0f83..67c44f2 100755
--- a/dev-tools/travis/travis-install.sh
+++ b/dev-tools/travis/travis-install.sh
@@ -26,17 +26,16 @@ TRAVIS_SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" 
&& pwd )
 pip install --user -r ${TRAVIS_SCRIPT_DIR}/requirements.txt
 pip3 install --user -r ${TRAVIS_SCRIPT_DIR}/requirements.txt
 
-cd ${STORM_SRC_ROOT_DIR}/storm-shaded-deps/
-python ${TRAVIS_SCRIPT_DIR}/save-logs.py "install-shade.txt" mvn clean install 
--batch-mode
+python ${TRAVIS_SCRIPT_DIR}/save-logs.py "storm-shaded-deps/install-shade.txt" 
mvn clean install --batch-mode -pl storm-shaded-deps -am
 BUILD_RET_VAL=$?
 if [[ "$BUILD_RET_VAL" != "0" ]];
 then
-  cat "install-shade.txt"
+  cat "storm-shaded-deps/install-shade.txt"
   exit ${BUILD_RET_VAL}
 fi
 
 cd ${STORM_SRC_ROOT_DIR}
-python ${TRAVIS_SCRIPT_DIR}/save-logs.py "install.txt" mvn clean install 
-DskipTests -Pnative,examples,externals '-P!include-shaded-deps' --batch-mode
+python ${TRAVIS_SCRIPT_DIR}/save-logs.py "install.txt" mvn clean install 
-DskipTests -Pnative,examples,externals -pl '!storm-shaded-deps' --batch-mode
 BUILD_RET_VAL=$?
 
 if [[ "$BUILD_RET_VAL" != "0" ]];
diff --git a/dev-tools/travis/travis-script.sh 
b/dev-tools/travis/travis-script.sh
index cc22a77..f7d582c 100755
--- a/dev-tools/travis/travis-script.sh
+++ b/dev-tools/travis/travis-script.sh
@@ -40,9 +40,9 @@ elif [ "$2" == "External" ]
 then
   if [ "$TRAVIS_JDK_VERSION" == "openjdk11" ]
   then 
-
TEST_MODULES='!storm-client,!storm-server,!storm-core,!storm-webapp,!external/storm-cassandra,!external/storm-hive,!external/storm-hdfs,!external/storm-hbase,!sql/storm-sql-external/storm-sql-hdfs,!external/storm-hdfs-blobstore'
+
TEST_MODULES='!storm-client,!storm-server,!storm-core,!storm-webapp,!storm-shaded-deps,!external/storm-cassandra,!external/storm-hive,!external/storm-hdfs,!external/storm-hbase,!sql/storm-sql-external/storm-sql-hdfs,!external/storm-hdfs-blobstore'
   else
-TEST_MODULES='!storm-client,!storm-server,!storm-core,!storm-webapp'
+
TEST_MODULES='!storm-client,!storm-server,!storm-core,!storm-webapp,!storm-shaded-deps'
   fi
 fi
 # We should be concerned that Travis CI could be very slow because it uses VM
@@ -50,7 +50,7 @@ export STORM_TEST_TIMEOUT_MS=15
 # Travis only has 3GB of memory, lets use 1GB for build, and 1.5GB for forked 
JVMs
 export MAVEN_OPTS="-Xmx1024m"
 
-mvn --batch-mode test -fae -Pnative,all-tests,examples,externals 
'-P!include-shaded-deps' -Prat -pl "$TEST_MODULES"
+mvn --batch-mode test -fae -Pnative,all-tests,examples,externals -Prat -pl 
"$TEST_MODULES"
 BUILD_RET_VAL=$?
 
 for dir in `find . -type d -and -wholename \*/target/\*-reports`;
diff --git a/pom.xml b/pom.xml
index ab1a7fa..989fedb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -336,6 +336,7 @@
 
 
 
+storm-shaded-deps
 storm-checkstyle
 storm-multilang/javascript
 storm-multilang/python
@@ -379,15 +380,6 @@
 
 
 
-include-shaded-deps
-   

[storm-site] branch asf-site updated (7729343 -> eed5baa)

2019-06-20 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git.


from 7729343  Rebuild content
 new 4ad57d9  Set copyright year to 2019 (credit: Julien Nioche 
)
 new 1b8dcd9  Rebuild content
 new eed5baa  Merge pull request #10 from srdo/pr-3029

The 63 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 _includes/footer.html | 2 +-
 content/2012/08/02/storm080-released.html | 2 +-
 content/2012/09/06/storm081-released.html | 2 +-
 content/2013/01/11/storm082-released.html | 2 +-
 content/2013/12/08/storm090-released.html | 2 +-
 content/2014/04/10/storm-logo-contest.html| 2 +-
 content/2014/04/17/logo-pforrest.html | 2 +-
 content/2014/04/17/logo-squinones.html| 2 +-
 content/2014/04/19/logo-ssuleman.html | 2 +-
 content/2014/04/21/logo-rmarshall.html| 2 +-
 content/2014/04/22/logo-zsayari.html  | 2 +-
 content/2014/04/23/logo-abartos.html  | 2 +-
 content/2014/04/27/logo-cboustead.html| 2 +-
 content/2014/04/27/logo-sasili.html   | 2 +-
 content/2014/04/29/logo-jlee1.html| 2 +-
 content/2014/04/29/logo-jlee2.html| 2 +-
 content/2014/04/29/logo-jlee3.html| 2 +-
 content/2014/05/27/round1-results.html| 2 +-
 content/2014/06/17/contest-results.html   | 2 +-
 content/2014/06/25/storm092-released.html | 2 +-
 content/2014/10/20/storm093-release-candidate.html| 2 +-
 content/2014/11/25/storm093-released.html | 2 +-
 content/2015/03/25/storm094-released.html | 2 +-
 content/2015/06/04/storm095-released.html | 2 +-
 content/2015/06/15/storm0100-beta-released.html   | 2 +-
 content/2015/11/05/storm0100-released.html| 2 +-
 content/2015/11/05/storm096-released.html | 2 +-
 content/2016/04/12/storm100-released.html | 2 +-
 content/2016/05/05/storm0101-released.html| 2 +-
 content/2016/05/06/storm101-released.html | 2 +-
 content/2016/08/10/storm102-released.html | 2 +-
 content/2016/09/07/storm097-released.html | 2 +-
 content/2016/09/14/storm0102-released.html| 2 +-
 content/2017/02/14/storm103-released.html | 2 +-
 content/2017/03/29/storm110-released.html | 2 +-
 content/2017/07/28/storm104-released.html | 2 +-
 content/2017/08/01/storm111-released.html | 2 +-
 content/2017/09/15/storm105-released.html | 2 +-
 content/2018/02/14/storm106-released.html | 2 +-
 content/2018/02/15/storm112-released.html | 2 +-
 content/2018/02/15/storm120-released.html | 2 +-
 content/2018/02/19/storm121-released.html | 2 +-
 content/2018/06/04/storm113-released.html | 2 +-
 content/2018/06/04/storm122-released.html | 2 +-
 content/2019/05/30/storm200-released.html | 2 +-
 content/Powered-By.html   | 2 +-
 content/about/deployment.html | 2 +-
 content/about/fault-tolerant.html | 2 +-
 content/about/free-and-open-source.html   | 2 +-
 content/about/guarantees-data-processing.html | 2 +-
 content/about/integrates.html | 2 +-
 content/about/multi-language.html | 2 +-
 content/about/scalable.html   | 2 +-
 content/about/simple-api.html | 2 +-
 content/contribute/BYLAWS.html| 2 +-
 content/contribute/Contributing-to-Storm.html

[storm] branch master updated: copyright year set to 2019

2019-06-20 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 3c2cbaa  copyright year set to 2019
 new 6af03d3  Merge pull request #3029 from jnioche/patch-1
3c2cbaa is described below

commit 3c2cbaa0d99e5861d5bafed733d8f2220c7b3f3b
Author: Julien Nioche 
AuthorDate: Tue Jun 18 06:48:57 2019 +0100

copyright year set to 2019
---
 docs/_includes/footer.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/_includes/footer.html b/docs/_includes/footer.html
index 3dde496..f7dc274 100644
--- a/docs/_includes/footer.html
+++ b/docs/_includes/footer.html
@@ -43,7 +43,7 @@
 

 
-Copyright © 2015 http://www.apache.org;>Apache Software Foundation. All Rights 
Reserved. 
+Copyright © 2019 http://www.apache.org;>Apache Software Foundation. All Rights 
Reserved. 
 Apache Storm, Apache, the Apache feather logo, and the 
Apache Storm project logos are trademarks of The Apache Software Foundation. 
 All other marks mentioned may be trademarks or 
registered trademarks of their respective owners.
 



[storm] branch master updated: Remove reference to storm-deploy, which looks dead. Link to download page on storm.apache.org instead of github for download links. Fix link to newbie issues in DEVELOPE

2019-06-06 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new b310340  Remove reference to storm-deploy, which looks dead. Link to 
download page on storm.apache.org instead of github for download links. Fix 
link to newbie issues in DEVELOPER.md
 new b9ab5e5  Merge pull request #3018 from srdo/doc-changes
b310340 is described below

commit b3103407ccd84596422733dba72cdb191844623d
Author: Stig Rohde Døssing 
AuthorDate: Sat Jun 1 11:39:30 2019 +0200

Remove reference to storm-deploy, which looks dead. Link to download page 
on storm.apache.org instead of github for download links. Fix link to newbie 
issues in DEVELOPER.md
---
 DEVELOPER.md   | 2 +-
 docs/Pacemaker.md  | 2 +-
 docs/Setting-up-a-Storm-cluster.md | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/DEVELOPER.md b/DEVELOPER.md
index 8fd8b83..e78 100644
--- a/DEVELOPER.md
+++ b/DEVELOPER.md
@@ -90,7 +90,7 @@ _If you are interested in contributing code to Storm but do 
not know where to be
 In this case you should
 [browse our issue tracker for open issues and 
tasks](https://issues.apache.org/jira/browse/STORM/?selectedTab=com.atlassian.jira.jira-projects-plugin:issues-panel).
 You may want to start with beginner-friendly, easier issues
-([newbie 
issues](https://issues.apache.org/jira/browse/STORM-58?jql=project%20%3D%20STORM%20AND%20labels%20%3D%20newbie%20AND%20status%20%3D%20Open)
+([newbie 
issues](https://issues.apache.org/jira/issues/?jql=project%20%3D%20STORM%20AND%20status%20%3D%20Open%20AND%20labels%20%3D%20newbie)
 and
 [trivial 
issues](https://issues.apache.org/jira/secure/IssueNavigator.jspa?reset=true=project+%3D+STORM+AND+resolution+%3D+Unresolved+AND+priority+%3D+Trivial+ORDER+BY+key+DESC=hide))
 because they require learning about only an isolated portion of the codebase 
and are a relatively small amount of work.
diff --git a/docs/Pacemaker.md b/docs/Pacemaker.md
index 5fc789c..75fa748 100644
--- a/docs/Pacemaker.md
+++ b/docs/Pacemaker.md
@@ -19,7 +19,7 @@ The corresponding Pacemaker client is a plugin for the 
`ClusterState` interface,
  - `pacemaker.servers` : The hosts that the Pacemaker daemons are running on
  - `pacemaker.port` : The port that Pacemaker will listen on
  - `pacemaker.max.threads` : Maximum number of threads Pacemaker daemon will 
use to handle requests.
- - `pacemaker.childopts` : Any JVM parameters that need to go to the 
Pacemaker. (used by storm-deploy project)
+ - `pacemaker.childopts` : Any JVM parameters that need to go to the 
Pacemaker. 
  - `pacemaker.auth.method` : The authentication method that is used (more info 
below)
 
  Example
diff --git a/docs/Setting-up-a-Storm-cluster.md 
b/docs/Setting-up-a-Storm-cluster.md
index d770a58..ea05da4 100644
--- a/docs/Setting-up-a-Storm-cluster.md
+++ b/docs/Setting-up-a-Storm-cluster.md
@@ -3,7 +3,7 @@ title: Setting up a Storm Cluster
 layout: documentation
 documentation: true
 ---
-This page outlines the steps for getting a Storm cluster up and running. If 
you're on AWS, you should check out the 
[storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) project. 
[storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) completely 
automates the provisioning, configuration, and installation of Storm clusters 
on EC2. It also sets up Ganglia for you so you can monitor CPU, disk, and 
network usage.
+This page outlines the steps for getting a Storm cluster up and running. 
 
 If you run into difficulties with your Storm cluster, first check for a 
solution is in the [Troubleshooting](Troubleshooting.html) page. Otherwise, 
email the mailing list.
 
@@ -37,7 +37,7 @@ These are the versions of the dependencies that have been 
tested with Storm. Sto
 
 ### Download and extract a Storm release to Nimbus and worker machines
 
-Next, download a Storm release and extract the zip file somewhere on Nimbus 
and each of the worker machines. The Storm releases can be downloaded [from 
here](http://github.com/apache/storm/releases).
+Next, download a Storm release and extract the zip file somewhere on Nimbus 
and each of the worker machines. The Storm releases can be downloaded [from 
here](../../downloads.html).
 
 ### Fill in mandatory configurations into storm.yaml
 



[storm] branch master updated: STORM-3401 Scan code quality at build time

2019-06-04 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4815bdc  STORM-3401 Scan code quality at build time
 new f4e18ab  Merge pull request #3019 from krichter722/checkstyle
4815bdc is described below

commit 4815bdc5ca5a2f9e8f94a4e62e439b9dd8003197
Author: Karl-Philipp Richter 
AuthorDate: Sat Jun 1 19:58:15 2019 +0200

STORM-3401 Scan code quality at build time

Use Maven checkstyle and PMD plugins during build rather than reporting 
phase in order to make validation obligatory
---
 examples/storm-elasticsearch-examples/pom.xml  |  4 
 examples/storm-hbase-examples/pom.xml  |  4 
 examples/storm-hdfs-examples/pom.xml   |  4 
 examples/storm-hive-examples/pom.xml   |  4 
 examples/storm-jdbc-examples/pom.xml   |  4 
 examples/storm-jms-examples/pom.xml|  4 
 examples/storm-kafka-client-examples/pom.xml   |  4 
 examples/storm-loadgen/pom.xml |  4 
 examples/storm-mongodb-examples/pom.xml|  4 
 examples/storm-mqtt-examples/pom.xml   |  4 
 examples/storm-opentsdb-examples/pom.xml   |  4 
 examples/storm-perf/pom.xml|  4 
 examples/storm-pmml-examples/pom.xml   |  4 
 examples/storm-redis-examples/pom.xml  |  4 
 examples/storm-rocketmq-examples/pom.xml   |  4 
 examples/storm-solr-examples/pom.xml   |  4 
 examples/storm-starter/pom.xml |  4 
 external/storm-autocreds/pom.xml   | 16 
 external/storm-blobstore-migration/pom.xml | 12 
 external/storm-cassandra/pom.xml   |  4 
 external/storm-elasticsearch/pom.xml   |  4 
 external/storm-eventhubs/pom.xml   |  4 
 external/storm-hbase/pom.xml   |  4 
 external/storm-hdfs-blobstore/pom.xml  |  4 
 external/storm-hdfs/pom.xml|  4 
 external/storm-hive/pom.xml|  4 
 external/storm-jdbc/pom.xml|  4 
 external/storm-jms/pom.xml |  4 
 external/storm-kafka-client/pom.xml|  4 
 external/storm-kafka-migration/pom.xml |  4 
 external/storm-kafka-monitor/pom.xml   |  4 
 external/storm-kinesis/pom.xml |  4 
 external/storm-metrics/pom.xml |  4 
 external/storm-mongodb/pom.xml |  4 
 external/storm-mqtt/pom.xml|  4 
 external/storm-opentsdb/pom.xml|  4 
 external/storm-pmml/pom.xml|  4 
 external/storm-redis/pom.xml   |  4 
 external/storm-rocketmq/pom.xml|  4 
 external/storm-solr/pom.xml|  4 
 flux/flux-core/pom.xml |  4 
 flux/flux-examples/pom.xml |  4 
 flux/flux-wrappers/pom.xml |  4 
 flux/pom.xml   | 13 +
 integration-test/pom.xml   | 12 
 pom.xml| 22 +-
 sql/pom.xml| 13 +
 sql/storm-sql-core/pom.xml |  4 
 sql/storm-sql-external/storm-sql-hdfs/pom.xml  |  4 
 sql/storm-sql-external/storm-sql-kafka/pom.xml |  4 
 sql/storm-sql-external/storm-sql-mongodb/pom.xml   |  4 
 sql/storm-sql-external/storm-sql-redis/pom.xml |  4 
 sql/storm-sql-runtime/pom.xml  |  4 
 .../maven-shade-clojure-transformer/pom.xml|  4 
 storm-buildtools/storm-maven-plugins/pom.xml   |  4 
 storm-client/pom.xml   |  8 
 storm-clojure-test/pom.xml |  8 
 storm-clojure/pom.xml  |  4 
 storm-core/pom.xml |  4 
 storm-multilang/javascript/pom.xml | 13 +
 storm-multilang/python/pom.xml | 13 +
 storm-multilang/ruby/pom.xml   | 13 +
 storm-server/pom.xml   |  4 
 storm-shaded-deps/pom.xml  |  8 
 storm-submit-tools/pom.xml |  4 
 storm-webapp/pom.xml   |  4 
 66 files changed, 350 insertions(+), 17 deletions(-)

diff --git a/examples/storm-elasticsearch-examples

[storm-site] branch asf-site updated (bb159a3 -> ec2177e)

2019-06-01 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git.


from bb159a3  Rebuild content
 new 2fd7e14  STORM-2949/STORM-3095: Remove MD5 links for new releases on 
downloads page, remove reference to storm-deploy which looks dead, make link on 
setting-up-cluster page for releases link to Storm releases page, rather than 
Github
 new 7191aa4  Rebuild content
 new ec2177e  Merge pull request #9 from srdo/STORM-3095-2949

The 58 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 about/deployment.md   |  2 --
 content/about/deployment.html |  2 --
 content/downloads.html| 15 ---
 content/feed.xml  |  4 ++--
 .../2.0.0-SNAPSHOT/Setting-up-a-Storm-cluster.html|  6 +++---
 downloads.html| 15 ---
 releases/2.0.0-SNAPSHOT/Setting-up-a-Storm-cluster.md |  6 +++---
 7 files changed, 8 insertions(+), 42 deletions(-)



[storm-site] 02/02: Rebuild content

2019-05-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit bb159a3ba019bb2106dd3cd252fd16128ef8fd94
Author: Stig Rohde Døssing 
AuthorDate: Thu May 30 23:25:11 2019 +0200

Rebuild content
---
 content/2019/05/30/storm200-released.html   | 27 +---
 content/feed.xml| 51 ++
 content/releases.html   | 56 -
 content/releases/1.2.2/metrics_v2.html  |  2 +-
 content/releases/2.0.0-SNAPSHOT/metrics_v2.html |  2 +-
 content/releases/current/metrics_v2.html|  2 +-
 6 files changed, 87 insertions(+), 53 deletions(-)

diff --git a/content/2019/05/30/storm200-released.html 
b/content/2019/05/30/storm200-released.html
index 5f93f20..fd5b6a0 100644
--- a/content/2019/05/30/storm200-released.html
+++ b/content/2019/05/30/storm200-released.html
@@ -296,22 +296,21 @@
 
 The most significant change to Storms Kafka integration since 1.x, is 
that storm-kafka has been removed. The module was deprecated a while back, due 
to Kafkas deprecation of the underlying client library. Users will have to 
move to the storm-kafka-client module, which uses Kafkas ´kafka-clients´ 
library for integration.
 
-For the most part, the migration to storm-kafka-client is straightforward. 
The documentation for storm-kafka-client contains a helpful mapping between the 
old and new spout configurations. If you are using any of the storm-kafka 
spouts, you will need to migrate offset checkpoints to the new spout, to avoid 
the new spout starting from scratch on your partitions. Storm provides a helper 
tool to do this which can be found https://github.com/apache/storm/tree/master/external/sto [...]
+For the most part, the migration to storm-kafka-client is straightforward. 
The documentation for storm-kafka-client contains a helpful mapping between the 
old and new spout configurations. If you are using any of the storm-kafka 
spouts, you will need to migrate offset checkpoints to the new spout, to avoid 
the new spout starting from scratch on your partitions. Storm provides a helper 
tool to do this which can be found https://github.com/apache/storm/tree/master/external/sto [...]
 
 When performing a migration, you should stop your topology, run the 
migration tool, then redeploy your topology with the storm-kafka-client 
spout.
 
 Move to Using the 
KafkaConsumer.assign API
 
-Storm-kafka-client in Storm 1.x allowed you to use Kafkas own 
mechanism to manage which spout tasks were responsible for which partitions. 
This mechanism was a poor fit for Storm, and was deprecated in 1.2.0. It has 
been https://issues.apache.org/jira/browse/STORM-2542.;>removed 
entirely in 2.0
+Storm-kafka-client in Storm 1.x allowed you to use Kafkas own 
mechanism to manage which spout tasks were responsible for which partitions. 
This mechanism was a poor fit for Storm, and was deprecated in 1.2.0. It has 
been https://issues.apache.org/jira/browse/STORM-2542;>removed 
entirely in 2.0.
 
 The storm-kafka-client Subscription interface has also been removed. It 
offered too limited control over the subscription behavior. It has been 
replaced with the TopicFilter and ManualPartitioner interfaces. Unless you were 
using a custom Subscription implementation, this will likely not
-affect you. If you were using a custom Subscription, https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md#manual-partition-assigment-advanced;>the
 storm-kafka-client documentation describes how to customize assignment
-.
+affect you. If you were using a custom Subscription, https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md#manual-partition-assigment-advanced;>the
 storm-kafka-client documentation describes how to customize assignment.
 
 Other Kafka Highlights
 
 
-The KafkaBolt now allows you to https://issues.apache.org/jira/browse/STORM-3175.;>specify a callback 
that will be called when a batch is written to Kafka
+The KafkaBolt now allows you to https://issues.apache.org/jira/browse/STORM-3175;>specify a callback that 
will be called when a batch is written to Kafka.
 The FirstPollOffsetStrategy behavior has been made consistent between the 
non-Trident and Trident spouts. It is now always the case that EARLIEST/LATEST 
only take effect on topology redeploy, and not when a worker restarts https://issues.apache.org/jira/browse/STORM-2990;>https://issues.apache.org/jira/browse/STORM-2990.
 Storm-kafka-client now has a transactional non-opaque Trident spout https://issues.apache.org/jira/browse/STORM-2974;>https://issues.apache.org/jira/browse/STORM-2974.
 There are new example modules for storm-kafka-client. You can find them https://github.com/apache/storm/tree/master/examples/storm-kafka-client-examples;>here.
@@ -322,6 +321,24 @@ affect you. If you were using a

[storm-site] 01/02: Fix some dead links and add parts to 2.0.0 release note about Java 8 and storm-core split, fix metrics v2 CsvStormReporter class name

2019-05-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git

commit 661030e994b5633855ab859f637225ce34b494a7
Author: Stig Rohde Døssing 
AuthorDate: Thu May 30 23:04:20 2019 +0200

Fix some dead links and add parts to 2.0.0 release note about Java 8 and 
storm-core split, fix metrics v2 CsvStormReporter class name
---
 _posts/2019-05-30-storm200-released.md | 31 ++-
 releases/1.2.2/metrics_v2.md   |  2 +-
 releases/2.0.0-SNAPSHOT/metrics_v2.md  |  2 +-
 3 files changed, 28 insertions(+), 7 deletions(-)

diff --git a/_posts/2019-05-30-storm200-released.md 
b/_posts/2019-05-30-storm200-released.md
index 414a873..38f1264 100644
--- a/_posts/2019-05-30-storm200-released.md
+++ b/_posts/2019-05-30-storm200-released.md
@@ -34,20 +34,19 @@ For more details see [stateful windowing 
documentation](https://github.com/apach
 ### Removal of Storm-Kafka
 The most significant change to Storm's Kafka integration since 1.x, is that 
storm-kafka has been removed. The module was deprecated a while back, due to 
Kafka's deprecation of the underlying client library. Users will have to move 
to the storm-kafka-client module, which uses Kafka's ´kafka-clients´ library 
for integration.
 
-For the most part, the migration to storm-kafka-client is straightforward. The 
documentation for storm-kafka-client contains a helpful mapping between the old 
and new spout configurations. If you are using any of the storm-kafka spouts, 
you will need to migrate offset checkpoints to the new spout, to avoid the new 
spout starting from scratch on your partitions. Storm provides a helper tool to 
do this which can be found 
[here](fhttps://github.com/apache/storm/tree/master/external/storm-ka [...]
+For the most part, the migration to storm-kafka-client is straightforward. The 
documentation for storm-kafka-client contains a helpful mapping between the old 
and new spout configurations. If you are using any of the storm-kafka spouts, 
you will need to migrate offset checkpoints to the new spout, to avoid the new 
spout starting from scratch on your partitions. Storm provides a helper tool to 
do this which can be found 
[here](fhttps://github.com/apache/storm/tree/master/external/storm-ka [...]
 
 When performing a migration, you should stop your topology, run the migration 
tool, then redeploy your topology with the storm-kafka-client spout.
 
 ### Move to Using the KafkaConsumer.assign API
-Storm-kafka-client in Storm 1.x allowed you to use Kafka's own mechanism to 
manage which spout tasks were responsible for which partitions. This mechanism 
was a poor fit for Storm, and was deprecated in 1.2.0. It has been [removed 
entirely in 2.0](https://issues.apache.org/jira/browse/STORM-2542.)
+Storm-kafka-client in Storm 1.x allowed you to use Kafka's own mechanism to 
manage which spout tasks were responsible for which partitions. This mechanism 
was a poor fit for Storm, and was deprecated in 1.2.0. It has been [removed 
entirely in 2.0](https://issues.apache.org/jira/browse/STORM-2542).
 
 The storm-kafka-client Subscription interface has also been removed. It 
offered too limited control over the subscription behavior. It has been 
replaced with the TopicFilter and ManualPartitioner interfaces. Unless you were 
using a custom Subscription implementation, this will likely not
-affect you. If you were using a custom Subscription, [the storm-kafka-client 
documentation describes how to customize 
assignment](https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md#manual-partition-assigment-advanced)
-.
+affect you. If you were using a custom Subscription, [the storm-kafka-client 
documentation describes how to customize 
assignment](https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md#manual-partition-assigment-advanced).
 
 
 ### Other Kafka Highlights
-* The KafkaBolt now allows you to [specify a callback that will be called when 
a batch is written to Kafka](https://issues.apache.org/jira/browse/STORM-3175.)
+* The KafkaBolt now allows you to [specify a callback that will be called when 
a batch is written to Kafka](https://issues.apache.org/jira/browse/STORM-3175).
 * The FirstPollOffsetStrategy behavior has been made consistent between the 
non-Trident and Trident spouts. It is now always the case that EARLIEST/LATEST 
only take effect on topology redeploy, and not when a worker restarts 
https://issues.apache.org/jira/browse/STORM-2990.
 * Storm-kafka-client now has a transactional non-opaque Trident spout 
https://issues.apache.org/jira/browse/STORM-2974.
 * There are new example modules for storm-kafka-client. You can find them 
[here](https://github.com/apache/storm/tree/master/examples/storm-kafka-client-examples).
@@ -56,6 +55,28 @@ affect you. If you were using a custom Subscription, [the 
storm-kafka-client doc
 ## EOL for 1.0.x
 With the release

[storm-site] branch asf-site updated (7814c42 -> bb159a3)

2019-05-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/storm-site.git.


from 7814c42  publish 2.0.0 release
 new 661030e  Fix some dead links and add parts to 2.0.0 release note about 
Java 8 and storm-core split, fix metrics v2 CsvStormReporter class name
 new bb159a3  Rebuild content

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 _posts/2019-05-30-storm200-released.md  | 31 +++---
 content/2019/05/30/storm200-released.html   | 27 +---
 content/feed.xml| 51 ++
 content/releases.html   | 56 -
 content/releases/1.2.2/metrics_v2.html  |  2 +-
 content/releases/2.0.0-SNAPSHOT/metrics_v2.html |  2 +-
 content/releases/current/metrics_v2.html|  2 +-
 releases/1.2.2/metrics_v2.md|  2 +-
 releases/2.0.0-SNAPSHOT/metrics_v2.md   |  2 +-
 9 files changed, 115 insertions(+), 60 deletions(-)



[storm] branch 1.x-branch updated: Fix class name of CsvStormReporter

2019-05-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/1.x-branch by this push:
 new 5b01c78  Fix class name of CsvStormReporter
5b01c78 is described below

commit 5b01c78d053b2b8e2b7e6a4a8acd4c822924bbf9
Author: Manuel Dossinger 
AuthorDate: Fri May 17 14:44:13 2019 +0200

Fix class name of CsvStormReporter
---
 docs/metrics_v2.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/metrics_v2.md b/docs/metrics_v2.md
index 8c7653f..ac2bfa7 100644
--- a/docs/metrics_v2.md
+++ b/docs/metrics_v2.md
@@ -76,7 +76,7 @@ public class TupleCountingBolt extends BaseRichBolt {
  
   * Console Reporter 
(`org.apache.storm.metrics2.reporters.ConsoleStormReporter`):
 Reports metrics to `System.out`.
-  * CSV Reporter (`org.apache.storm.metrics2.reporters.CsvReporter`):
+  * CSV Reporter (`org.apache.storm.metrics2.reporters.CsvStormReporter`):
 Reports metrics to a CSV file.
   * Graphite Reporter 
(`org.apache.storm.metrics2.reporters.GraphiteStormReporter`):
 Reports metrics to a [Graphite](https://graphiteapp.org) server.



[storm] branch master updated: Fix class name of CsvStormReporter

2019-05-30 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 4fcce90  Fix class name of CsvStormReporter
 new 92008a2  Merge pull request #3009 from contradictioned/master
4fcce90 is described below

commit 4fcce90f77b17554ec3d3940b4af98ad1b52f006
Author: Manuel Dossinger 
AuthorDate: Fri May 17 14:44:13 2019 +0200

Fix class name of CsvStormReporter
---
 docs/metrics_v2.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/metrics_v2.md b/docs/metrics_v2.md
index ad52b39..e89e4e3 100644
--- a/docs/metrics_v2.md
+++ b/docs/metrics_v2.md
@@ -76,7 +76,7 @@ public class TupleCountingBolt extends BaseRichBolt {
  
   * Console Reporter 
(`org.apache.storm.metrics2.reporters.ConsoleStormReporter`):
 Reports metrics to `System.out`.
-  * CSV Reporter (`org.apache.storm.metrics2.reporters.CsvReporter`):
+  * CSV Reporter (`org.apache.storm.metrics2.reporters.CsvStormReporter`):
 Reports metrics to a CSV file.
   * Graphite Reporter 
(`org.apache.storm.metrics2.reporters.GraphiteStormReporter`):
 Reports metrics to a [Graphite](https://graphiteapp.org) server.



[storm] branch master updated: STORM-3390: Lock python test dependency versions

2019-05-08 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new d3b8da0  STORM-3390: Lock python test dependency versions
 new ecf97fc  Merge branch 'STORM-3390' of https://github.com/srdo/storm 
into asfgit-master
d3b8da0 is described below

commit d3b8da09bad45975d85c035ae9e9d26836e0a7ea
Author: Stig Rohde Døssing 
AuthorDate: Sun May 5 14:43:37 2019 +0200

STORM-3390: Lock python test dependency versions
---
 dev-tools/travis/requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-tools/travis/requirements.txt 
b/dev-tools/travis/requirements.txt
index 51f6f05..b3ff2de 100644
--- a/dev-tools/travis/requirements.txt
+++ b/dev-tools/travis/requirements.txt
@@ -10,4 +10,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-mock
\ No newline at end of file
+mock == 2.0.0
\ No newline at end of file



[storm] 04/04: Merge branch 'STORM-3379' of https://github.com/srdo/storm into asfgit-master

2019-05-05 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git

commit 26e6c06c4b94aa121457889b1bd22c74fa41b3f7
Merge: 2bf972e c985695
Author: Stig Rohde Døssing 
AuthorDate: Sun May 5 16:13:19 2019 +0200

Merge branch 'STORM-3379' of https://github.com/srdo/storm into 
asfgit-master

 .../jvm/org/apache/storm/daemon/worker/Worker.java |  26 +++--
 .../apache/storm/daemon/worker/WorkerState.java|  45 +
 .../auth/workertoken/WorkerTokenAuthorizer.java|   2 +-
 .../org/apache/storm/utils/SupervisorClient.java   |   5 +-
 .../apache/storm/utils/SupervisorIfaceFactory.java |  30 ++
 .../apache/storm/daemon/supervisor/Container.java  |   4 +-
 .../storm/daemon/supervisor/ContainerLauncher.java |   7 +-
 .../storm/daemon/supervisor/LocalContainer.java|  10 +-
 .../daemon/supervisor/LocalContainerLauncher.java  |   7 +-
 .../storm/daemon/supervisor/ReadClusterState.java  |   3 +-
 .../apache/storm/daemon/supervisor/Supervisor.java | 108 -
 .../nimbus/AssignmentDistributionService.java  |   2 +-
 12 files changed, 164 insertions(+), 85 deletions(-)



[storm] 03/04: Merge branch 'STORM-3376' of https://github.com/srdo/storm into asfgit-master

2019-05-05 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git

commit 2bf972e050af2356940f2b9ce59af6d660f49766
Merge: 8951e7a aaf1113
Author: Stig Rohde Døssing 
AuthorDate: Sun May 5 16:12:55 2019 +0200

Merge branch 'STORM-3376' of https://github.com/srdo/storm into 
asfgit-master

 integration-test/config/storm.yaml |  6 ++
 .../test/org/apache/storm/sql/TestStormSql.java| 45 +-
 .../backends/streams/TestPlanCompiler.java | 96 ++---
 .../src/test/org/apache/storm/sql/TestUtils.java   | 23 ++---
 .../org/apache/storm/cluster/IStateStorage.java|  4 +-
 .../storm/cluster/StormClusterStateImpl.java   |  8 +-
 .../jvm/org/apache/storm/daemon/worker/Worker.java |  4 +-
 .../apache/storm/daemon/worker/WorkerState.java| 40 -
 .../org/apache/storm/messaging/IConnection.java| 14 
 .../jvm/org/apache/storm/messaging/IContext.java   |  5 +-
 .../org/apache/storm/messaging/local/Context.java  | 98 ++
 .../org/apache/storm/messaging/netty/Client.java   | 15 
 .../org/apache/storm/messaging/netty/Context.java  |  6 +-
 .../org/apache/storm/messaging/netty/Server.java   | 29 +++
 .../apache/storm/security/auth/ThriftServer.java   |  8 +-
 .../auth/digest/DigestSaslTransportPlugin.java | 11 ++-
 .../auth/kerberos/KerberosSaslTransportPlugin.java | 11 ++-
 .../security/auth/sasl/SaslTransportPlugin.java| 11 ++-
 .../auth/workertoken/WorkerTokenAuthorizer.java| 13 ++-
 .../apache/storm/messaging/netty/NettyTest.java| 28 +++
 .../main/java/org/apache/storm/LocalCluster.java   |  6 +-
 .../apache/storm/blobstore/LocalFsBlobStore.java   |  1 +
 .../org/apache/storm/daemon/supervisor/Slot.java   |  2 +-
 .../org/apache/storm/localizer/AsyncLocalizer.java |  4 +-
 .../security/auth/workertoken/WorkerTokenTest.java | 24 +++---
 25 files changed, 247 insertions(+), 265 deletions(-)



[storm] branch master updated (8951e7a -> 26e6c06)

2019-05-05 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git.


from 8951e7a  Merge branch 'STORM-3373' of https://github.com/srdo/storm 
into STORM-3373-merge
 new aaf1113  STORM-3376: Set Server callback before opening Netty socket, 
so we don't drop messages during startup
 new 2bf972e  Merge branch 'STORM-3376' of https://github.com/srdo/storm 
into asfgit-master
 new c985695  STORM-3379: Fix intermittent NPE during worker boot in local 
mode
 new 26e6c06  Merge branch 'STORM-3379' of https://github.com/srdo/storm 
into asfgit-master

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 integration-test/config/storm.yaml |   6 ++
 .../test/org/apache/storm/sql/TestStormSql.java|  45 -
 .../backends/streams/TestPlanCompiler.java |  96 +-
 .../src/test/org/apache/storm/sql/TestUtils.java   |  23 ++---
 .../org/apache/storm/cluster/IStateStorage.java|   4 +-
 .../storm/cluster/StormClusterStateImpl.java   |   8 +-
 .../jvm/org/apache/storm/daemon/worker/Worker.java |  30 --
 .../apache/storm/daemon/worker/WorkerState.java|  83 
 .../org/apache/storm/messaging/IConnection.java|  14 ---
 .../jvm/org/apache/storm/messaging/IContext.java   |   5 +-
 .../org/apache/storm/messaging/local/Context.java  |  98 +--
 .../org/apache/storm/messaging/netty/Client.java   |  15 ---
 .../org/apache/storm/messaging/netty/Context.java  |   6 +-
 .../org/apache/storm/messaging/netty/Server.java   |  29 +++---
 .../apache/storm/security/auth/ThriftServer.java   |   8 +-
 .../auth/digest/DigestSaslTransportPlugin.java |  11 ++-
 .../auth/kerberos/KerberosSaslTransportPlugin.java |  11 ++-
 .../security/auth/sasl/SaslTransportPlugin.java|  11 ++-
 .../auth/workertoken/WorkerTokenAuthorizer.java|  15 ++-
 .../org/apache/storm/utils/SupervisorClient.java   |   5 +-
 .../apache/storm/utils/SupervisorIfaceFactory.java |  13 ++-
 .../apache/storm/messaging/netty/NettyTest.java|  28 ++
 .../main/java/org/apache/storm/LocalCluster.java   |   6 +-
 .../apache/storm/blobstore/LocalFsBlobStore.java   |   1 +
 .../apache/storm/daemon/supervisor/Container.java  |   4 +-
 .../storm/daemon/supervisor/ContainerLauncher.java |   7 +-
 .../storm/daemon/supervisor/LocalContainer.java|  10 +-
 .../daemon/supervisor/LocalContainerLauncher.java  |   7 +-
 .../storm/daemon/supervisor/ReadClusterState.java  |   3 +-
 .../org/apache/storm/daemon/supervisor/Slot.java   |   2 +-
 .../apache/storm/daemon/supervisor/Supervisor.java | 108 -
 .../org/apache/storm/localizer/AsyncLocalizer.java |   4 +-
 .../nimbus/AssignmentDistributionService.java  |   2 +-
 .../security/auth/workertoken/WorkerTokenTest.java |  24 ++---
 34 files changed, 389 insertions(+), 353 deletions(-)
 copy 
storm-server/src/main/java/org/apache/storm/daemon/drpc/RequestFactory.java => 
storm-client/src/jvm/org/apache/storm/utils/SupervisorIfaceFactory.java (78%)



[storm] branch master updated: STORM-3386: Require Maven version 3.5.0 for Storm's build

2019-05-03 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 0fcbb4e  STORM-3386: Require Maven version 3.5.0 for Storm's build
 new 1c94048  Merge branch 'STORM-3386' of https://github.com/srdo/storm 
into asfgit-master
0fcbb4e is described below

commit 0fcbb4e747ab25ae5435827a6aff054c1efe6be1
Author: Stig Rohde Døssing 
AuthorDate: Wed May 1 22:36:18 2019 +0200

STORM-3386: Require Maven version 3.5.0 for Storm's build
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 4bb35f8..27fcc6c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1436,7 +1436,7 @@
 
 
 
-3.0
+3.5
 
 
 



[storm] branch 1.1.x-branch updated: [STORM-3384] set-log-level command throws wrong exception if the topology is not running

2019-04-26 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 1.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/1.1.x-branch by this push:
 new 3f783c8  [STORM-3384] set-log-level command throws wrong exception if 
the topology is not running
 new 164fda9  Merge branch 'STORM-3384-1.1.x' of 
https://github.com/Ethanlm/storm into asfgit-1.1.x-branch
3f783c8 is described below

commit 3f783c8c82906e4688b8d5d19b7b0b3013eb2cda
Author: Ethan Li 
AuthorDate: Thu Apr 25 09:27:18 2019 -0500

[STORM-3384] set-log-level command throws wrong exception if the topology 
is not running
---
 storm-core/src/clj/org/apache/storm/command/set_log_level.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/storm-core/src/clj/org/apache/storm/command/set_log_level.clj 
b/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
index 7e1c3c5..27dc4c5 100644
--- a/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
+++ b/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
@@ -28,7 +28,7 @@
 topology (first (filter (fn [topo] (= name (.get_name topo))) 
topologies))]
 (if topology 
   (.get_id topology)
-  (throw (.IllegalArgumentException (str name " is not a running 
topology"))
+  (throw (IllegalArgumentException. (str name " is not a running 
topology"))
 
 (defn- parse-named-log-levels [action]
   "Parses [logger name]=[level string]:[optional timeout],[logger name2]...



[storm] branch 1.x-branch updated: [STORM-3384] set-log-level command throws wrong exception if the topology is not running

2019-04-26 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/1.x-branch by this push:
 new cad06ec  [STORM-3384] set-log-level command throws wrong exception if 
the topology is not running
 new 194c42a  Merge branch 'STORM-3384-1.x' of 
https://github.com/Ethanlm/storm into asfgit-1.x-branch
cad06ec is described below

commit cad06ecc578fabde696d01655b4d26fddc057f04
Author: Ethan Li 
AuthorDate: Thu Apr 25 09:27:18 2019 -0500

[STORM-3384] set-log-level command throws wrong exception if the topology 
is not running
---
 storm-core/src/clj/org/apache/storm/command/set_log_level.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/storm-core/src/clj/org/apache/storm/command/set_log_level.clj 
b/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
index 7e1c3c5..27dc4c5 100644
--- a/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
+++ b/storm-core/src/clj/org/apache/storm/command/set_log_level.clj
@@ -28,7 +28,7 @@
 topology (first (filter (fn [topo] (= name (.get_name topo))) 
topologies))]
 (if topology 
   (.get_id topology)
-  (throw (.IllegalArgumentException (str name " is not a running 
topology"))
+  (throw (IllegalArgumentException. (str name " is not a running 
topology"))
 
 (defn- parse-named-log-levels [action]
   "Parses [logger name]=[level string]:[optional timeout],[logger name2]...



[storm] branch 1.1.x-branch updated: STORM-3381: Exclude spotbugs-annotations from Zookeeper

2019-04-18 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 1.1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/1.1.x-branch by this push:
 new 84e6c95  STORM-3381: Exclude spotbugs-annotations from Zookeeper
84e6c95 is described below

commit 84e6c95f943e967bf53c43af4a261c72684fb13b
Author: Stig Rohde Døssing 
AuthorDate: Wed Apr 17 13:08:01 2019 +0200

STORM-3381: Exclude spotbugs-annotations from Zookeeper
---
 pom.xml| 4 
 storm-core/pom.xml | 4 
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/pom.xml b/pom.xml
index 654acf8..352f9a8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -660,6 +660,10 @@
 log4j
 log4j
 
+
+com.github.spotbugs
+spotbugs-annotations
+
 
 
 
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index cb0819d..8277800 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -275,10 +275,6 @@
 test
 
 
-org.apache.zookeeper
-zookeeper
-
-
 javax.servlet
 servlet-api
 



[storm] branch 1.x-branch updated: STORM-3381: Exclude spotbugs-annotations from Zookeeper

2019-04-18 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch 1.x-branch
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/1.x-branch by this push:
 new eb7f53e  STORM-3381: Exclude spotbugs-annotations from Zookeeper
eb7f53e is described below

commit eb7f53e7404dec462dc77963296fe2cc00932a99
Author: Stig Rohde Døssing 
AuthorDate: Wed Apr 17 13:08:01 2019 +0200

STORM-3381: Exclude spotbugs-annotations from Zookeeper
---
 pom.xml| 4 
 storm-core/pom.xml | 4 
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/pom.xml b/pom.xml
index 8ad193f..69e1b6f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -662,6 +662,10 @@
 log4j
 log4j
 
+
+com.github.spotbugs
+spotbugs-annotations
+
 
 
 
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 2a92e27..2793891 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -284,10 +284,6 @@
 test
 
 
-org.apache.zookeeper
-zookeeper
-
-
 javax.servlet
 servlet-api
 



[storm] branch master updated: STORM-3381: Exclude spotbugs-annotations from Zookeeper, regen DEPENDENCY-LICENSES

2019-04-18 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 263c8d3  STORM-3381: Exclude spotbugs-annotations from Zookeeper, 
regen DEPENDENCY-LICENSES
 new ef240c3  Merge branch 'STORM-3381' of https://github.com/srdo/storm 
into asfgit-master
263c8d3 is described below

commit 263c8d3562558f6ffdd6bbd208f7bed7b3f1370f
Author: Stig Rohde Døssing 
AuthorDate: Wed Apr 17 13:08:01 2019 +0200

STORM-3381: Exclude spotbugs-annotations from Zookeeper, regen 
DEPENDENCY-LICENSES
---
 DEPENDENCY-LICENSES   | 32 ++--
 pom.xml   |  5 -
 storm-shaded-deps/pom.xml |  4 
 3 files changed, 18 insertions(+), 23 deletions(-)

diff --git a/DEPENDENCY-LICENSES b/DEPENDENCY-LICENSES
index 2acd750..16cc35a 100644
--- a/DEPENDENCY-LICENSES
+++ b/DEPENDENCY-LICENSES
@@ -108,14 +108,12 @@ List of third-party dependencies grouped by their license 
type.
 * Apache HBase - Shaded Protocol 
(org.apache.hbase:hbase-protocol-shaded:2.1.3 - 
http://hbase.apache.org/hbase-build-configuration/hbase-protocol-shaded)
 * Apache HBase - Zookeeper (org.apache.hbase:hbase-zookeeper:2.1.3 - 
http://hbase.apache.org/hbase-build-configuration/hbase-zookeeper)
 * Apache HttpAsyncClient 
(org.apache.httpcomponents:httpasyncclient:4.1.2 - 
http://hc.apache.org/httpcomponents-asyncclient)
-* Apache HttpClient (org.apache.httpcomponents:httpclient:4.3.5 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient (org.apache.httpcomponents:httpclient:4.3.6 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient (org.apache.httpcomponents:httpclient:4.4.1 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient (org.apache.httpcomponents:httpclient:4.5.2 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient (org.apache.httpcomponents:httpclient:4.5.6 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient (org.apache.httpcomponents:httpclient:4.5 - 
http://hc.apache.org/httpcomponents-client)
 * Apache HttpClient Mime (org.apache.httpcomponents:httpmime:4.4.1 - 
http://hc.apache.org/httpcomponents-client)
-* Apache HttpCore (org.apache.httpcomponents:httpcore:4.3.2 - 
http://hc.apache.org/httpcomponents-core-ga)
 * Apache HttpCore (org.apache.httpcomponents:httpcore:4.3.3 - 
http://hc.apache.org/httpcomponents-core-ga)
 * Apache HttpCore (org.apache.httpcomponents:httpcore:4.4.10 - 
http://hc.apache.org/httpcomponents-core-ga)
 * Apache HttpCore (org.apache.httpcomponents:httpcore:4.4.1 - 
http://hc.apache.org/httpcomponents-core-ga)
@@ -124,7 +122,6 @@ List of third-party dependencies grouped by their license 
type.
 * Apache HttpCore NIO (org.apache.httpcomponents:httpcore-nio:4.4.5 - 
http://hc.apache.org/httpcomponents-core-ga)
 * Apache Ivy (org.apache.ivy:ivy:2.4.0 - http://ant.apache.org/ivy/)
 * Apache Kafka (org.apache.kafka:kafka-clients:0.11.0.3 - 
http://kafka.apache.org)
-* Apache Log4j (log4j:log4j:1.2.16 - 
http://logging.apache.org/log4j/1.2/)
 * Apache Log4j (log4j:log4j:1.2.17 - 
http://logging.apache.org/log4j/1.2/)
 * Apache Log4j 1.x Compatibility API 
(org.apache.logging.log4j:log4j-1.2-api:2.6.2 - 
http://logging.apache.org/log4j/2.x/log4j-1.2-api/)
 * Apache Log4j API (org.apache.logging.log4j:log4j-api:2.11.1 - 
https://logging.apache.org/log4j/2.x/log4j-api/)
@@ -452,27 +449,31 @@ List of third-party dependencies grouped by their license 
type.
 * Lucene Suggest (org.apache.lucene:lucene-suggest:5.5.2 - 
http://lucene.apache.org/lucene-parent/lucene-suggest)
 * LZ4 and xxHash (net.jpountz.lz4:lz4:1.3.0 - 
https://github.com/jpountz/lz4-java)
 * Maven Aether Provider (org.apache.maven:maven-aether-provider:3.0 - 
http://maven.apache.org/maven-aether-provider/)
-* Maven Aether Provider (org.apache.maven:maven-aether-provider:3.1.0 
- http://maven.apache.org/ref/3.1.0/maven-aether-provider)
 * Maven Artifact (org.apache.maven:maven-artifact:3.0 - 
http://maven.apache.org/maven-artifact/)
+* Maven Artifact (org.apache.maven:maven-artifact:3.6.0 - 
https://maven.apache.org/ref/3.6.0/maven-artifact/)
 * Maven Artifact Resolver API 
(org.apache.maven.resolver:maven-resolver-api:1.3.1 - 
https://maven.apache.org/resolver/maven-resolver-api/)
+* Maven Artifact Resolver API 
(org.apache.maven.resolver:maven-resolver-api:1.3.3 - 
https://maven.apache.org/resolver/maven-resolver-api/)
+* Maven Artifact Resolver Connector Basic 
(org.apache.maven.resolver:maven-resolver-connector-basic:1.3.3 - 
https://maven.apache.org/resolver/maven-resolver-connector-basic/)
 * Maven Artifact Resolver

[storm] branch master updated: [STORM-3375] change Date.getTime() to System.currentTimeMillis()

2019-04-16 Thread srdo
This is an automated email from the ASF dual-hosted git repository.

srdo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
 new 362b3b6  [STORM-3375] change Date.getTime() to 
System.currentTimeMillis()
 new b48e105  Merge branch 'STORM-3375' of 
https://github.com/bd2019us/storm into asfgit-master
362b3b6 is described below

commit 362b3b6a41c784eccc2f9362b403c45623a3ea51
Author: bd2019us 
AuthorDate: Thu Apr 11 23:40:58 2019 -0500

[STORM-3375] change Date.getTime() to System.currentTimeMillis()
---
 .../org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java   | 3 +--
 .../java/org/apache/storm/hive/bolt/mapper/JsonRecordHiveMapper.java   | 3 +--
 2 files changed, 2 insertions(+), 4 deletions(-)

diff --git 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
index d67ec67..f6e3612 100644
--- 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
+++ 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/DelimitedRecordHiveMapper.java
@@ -132,8 +132,7 @@ public class DelimitedRecordHiveMapper implements 
HiveMapper {
 }
 
 private String getPartitionsByTimeFormat() {
-Date d = new Date();
-return parseDate.format(d.getTime());
+return parseDate.format(System.currentTimeMillis());
 }
 
 @VisibleForTesting
diff --git 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/JsonRecordHiveMapper.java
 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/JsonRecordHiveMapper.java
index 3a43b7c..59c1bb7 100644
--- 
a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/JsonRecordHiveMapper.java
+++ 
b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/mapper/JsonRecordHiveMapper.java
@@ -119,7 +119,6 @@ public class JsonRecordHiveMapper implements HiveMapper {
 }
 
 private String getPartitionsByTimeFormat() {
-Date d = new Date();
-return parseDate.format(d.getTime());
+return parseDate.format(System.currentTimeMillis());
 }
 }



  1   2   3   4   5   6   7   8   9   10   >