[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370235#comment-15370235
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205892
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
in

[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205892
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
intervalToTaskToMetricToRegistry;
+protected final Map> streamToComponentToGrouper;
+protected final ReportErrorAndDie reportErrorDie;
+protected final Callable sampler;
+protected final AtomicBoolean backpressure;
+pr

[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370232#comment-15370232
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205841
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
in

[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205841
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
intervalToTaskToMetricToRegistry;
+protected final Map> streamToComponentToGrouper;
+protected final ReportErrorAndDie reportErrorDie;
+protected final Callable sampler;
+protected final AtomicBoolean backpressure;
+pr

[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370228#comment-15370228
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205731
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Task.java ---
@@ -179,23 +180,22 @@ public BuiltinMetrics getBuiltInMetrics() {
 private TopologyContext mkTopologyContext(StormTopology topology) 
throws IOException {
 Map conf = (Map) workerData.get("conf");
 return new TopologyContext(
-topology,
-(Map) workerData.get("storm-conf"),
-(Map) workerData.get("task->component"),
-(Map>) 
workerData.get("component->sorted-tasks"),
-(Map>) 
workerData.get("component->stream->fields"),
-(String) workerData.get("storm-id"),
-
ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf,
 (String) workerData.get("storm-id"))),
-ConfigUtils.workerPidsRoot(conf, (String) 
workerData.get("worker-id")),
-taskId,
-(Integer) workerData.get("port"),
-(List) workerData.get("task-ids"),
-(Map) 
workerData.get("default-shared-resources"),
-(Map) workerData.get("user-shared-resources"),
-(Map) executorData.get("shared-executor-data"),
-(Map>>) 
executorData.get("interval->task->metric-registry"),
-(clojure.lang.Atom) 
executorData.get("open-or-prepare-was-called?")
-);
+topology,
+(Map) workerData.get("storm-conf"),
--- End diff --

addressed, moved them into Constants.java


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370229#comment-15370229
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205741
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
--- End diff --

ok, addressed


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205741
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
--- End diff --

ok, addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205731
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Task.java ---
@@ -179,23 +180,22 @@ public BuiltinMetrics getBuiltInMetrics() {
 private TopologyContext mkTopologyContext(StormTopology topology) 
throws IOException {
 Map conf = (Map) workerData.get("conf");
 return new TopologyContext(
-topology,
-(Map) workerData.get("storm-conf"),
-(Map) workerData.get("task->component"),
-(Map>) 
workerData.get("component->sorted-tasks"),
-(Map>) 
workerData.get("component->stream->fields"),
-(String) workerData.get("storm-id"),
-
ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf,
 (String) workerData.get("storm-id"))),
-ConfigUtils.workerPidsRoot(conf, (String) 
workerData.get("worker-id")),
-taskId,
-(Integer) workerData.get("port"),
-(List) workerData.get("task-ids"),
-(Map) 
workerData.get("default-shared-resources"),
-(Map) workerData.get("user-shared-resources"),
-(Map) executorData.get("shared-executor-data"),
-(Map>>) 
executorData.get("interval->task->metric-registry"),
-(clojure.lang.Atom) 
executorData.get("open-or-prepare-was-called?")
-);
+topology,
+(Map) workerData.get("storm-conf"),
--- End diff --

addressed, moved them into Constants.java


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205708
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -539,10 +542,34 @@ protected IAuthorizer 
mkAuthorizationHandlerImpl(String klassName, Map conf) thr
 if (aznHandler != null) {
 aznHandler.prepare(conf);
 }
-LOG.debug("authorization class name:{}, class:{}, 
handler:{}",klassName, aznClass, aznHandler);
+LOG.debug("authorization class name:{}, class:{}, 
handler:{}", klassName, aznClass, aznHandler);
 }
 }
 
 return aznHandler;
 }
+
+public static WorkerTopologyContext makeWorkerContext(Map workerData) {
+try {
+StormTopology stormTopology = (StormTopology) 
workerData.get("system-topology");
--- End diff --

addressed, moved them into Constants class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370226#comment-15370226
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205708
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -539,10 +542,34 @@ protected IAuthorizer 
mkAuthorizationHandlerImpl(String klassName, Map conf) thr
 if (aznHandler != null) {
 aznHandler.prepare(conf);
 }
-LOG.debug("authorization class name:{}, class:{}, 
handler:{}",klassName, aznClass, aznHandler);
+LOG.debug("authorization class name:{}, class:{}, 
handler:{}", klassName, aznClass, aznHandler);
 }
 }
 
 return aznHandler;
 }
+
+public static WorkerTopologyContext makeWorkerContext(Map workerData) {
+try {
+StormTopology stormTopology = (StormTopology) 
workerData.get("system-topology");
--- End diff --

addressed, moved them into Constants class


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370220#comment-15370220
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205623
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Acker.java ---
@@ -101,7 +101,7 @@ public void execute(Tuple input) {
 }
 curr.failed = true;
 pending.put(id, curr);
-} else if (ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+} else if(ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
--- End diff --

addressed


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370222#comment-15370222
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205629
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -77,7 +78,7 @@ public static StormCommon setInstance(StormCommon common) 
{
 return oldInstance;
 }
 
-private static final Logger LOG = 
LoggerFactory.getLogger(StormCommon.class);
+private static final Logger LOG = getLogger(StormCommon.class);
--- End diff --

addressed


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205629
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -77,7 +78,7 @@ public static StormCommon setInstance(StormCommon common) 
{
 return oldInstance;
 }
 
-private static final Logger LOG = 
LoggerFactory.getLogger(StormCommon.class);
+private static final Logger LOG = getLogger(StormCommon.class);
--- End diff --

addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70205623
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Acker.java ---
@@ -101,7 +101,7 @@ public void execute(Tuple input) {
 }
 curr.failed = true;
 pending.put(id, curr);
-} else if (ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+} else if(ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
--- End diff --

addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370186#comment-15370186
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70203481
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
in

[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread unsleepy22
Github user unsleepy22 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70203481
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
intervalToTaskToMetricToRegistry;
+protected final Map> streamToComponentToGrouper;
+protected final ReportErrorAndDie reportErrorDie;
+protected final Callable sampler;
+protected final AtomicBoolean backpressure;
+pr

[jira] [Resolved] (STORM-1950) Change response json of "Topology Lag" REST API to keyed by spoutId, topic, partition

2016-07-10 Thread Jungtaek Lim (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1950?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jungtaek Lim resolved STORM-1950.
-
   Resolution: Fixed
Fix Version/s: 1.1.0
   2.0.0

Merged by [~sriharsha]

> Change response json of "Topology Lag" REST API to keyed by spoutId, topic, 
> partition
> -
>
> Key: STORM-1950
> URL: https://issues.apache.org/jira/browse/STORM-1950
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-ui
>Affects Versions: 1.1.0
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
> Fix For: 2.0.0, 1.1.0
>
>
> From code review for STORM-1945, there's an idea to change JSON response of 
> "Topology Lag" API to keyed by topic, partition number.
> https://github.com/apache/storm/pull/1541#issuecomment-230983140
> I think also make result keyed by spout id would be good.
> Here's sample JSON of output after this issue is resolved.
> {code}
> {
>"spout1":{
>   "spoutId":"spout1",
>   "spoutType":"KAFKA",
>   "spoutLagResult":{
>  "topic":{
> "partition0":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> },
> "partition2":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> }
>  },
>  "topic2":{
> "partition0":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> },
> "partition2":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> }
>  }
>   }
>}
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (STORM-1933) Intermittent test failure on test-multiple-active-storms-multiple-supervisors for supervisor-test

2016-07-10 Thread Jungtaek Lim (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jungtaek Lim resolved STORM-1933.
-
   Resolution: Fixed
Fix Version/s: 1.1.0
   1.0.2
   2.0.0

> Intermittent test failure on test-multiple-active-storms-multiple-supervisors 
> for supervisor-test 
> --
>
> Key: STORM-1933
> URL: https://issues.apache.org/jira/browse/STORM-1933
> Project: Apache Storm
>  Issue Type: Sub-task
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
> Attachments: 
> only-thread-1362-and-1363-BUG-60850-intermittent-failure-supervisor-test.txt
>
>
> test-multiple-active-storms-multiple-supervisors is failing with fairly high 
> chance. I've run unit test of 1.x branch 3 times and met this issue, and 
> users report FileNotFound issue on supervisor which seems to be related to 
> this.
> I have log file so I'll attach once issue is created.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1933) Intermittent test failure on test-multiple-active-storms-multiple-supervisors for supervisor-test

2016-07-10 Thread Jungtaek Lim (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370104#comment-15370104
 ] 

Jungtaek Lim commented on STORM-1933:
-

Intermittent failure is not shown within 20 times build. Closing.

> Intermittent test failure on test-multiple-active-storms-multiple-supervisors 
> for supervisor-test 
> --
>
> Key: STORM-1933
> URL: https://issues.apache.org/jira/browse/STORM-1933
> Project: Apache Storm
>  Issue Type: Sub-task
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
> Attachments: 
> only-thread-1362-and-1363-BUG-60850-intermittent-failure-supervisor-test.txt
>
>
> test-multiple-active-storms-multiple-supervisors is failing with fairly high 
> chance. I've run unit test of 1.x branch 3 times and met this issue, and 
> users report FileNotFound issue on supervisor which seems to be related to 
> this.
> I have log file so I'll attach once issue is created.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194730
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -77,7 +78,7 @@ public static StormCommon setInstance(StormCommon common) 
{
 return oldInstance;
 }
 
-private static final Logger LOG = 
LoggerFactory.getLogger(StormCommon.class);
+private static final Logger LOG = getLogger(StormCommon.class);
--- End diff --

lets keep it to LoggerFactory.getLogger given that we are using the same 
way everywhere else.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370079#comment-15370079
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194730
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -77,7 +78,7 @@ public static StormCommon setInstance(StormCommon common) 
{
 return oldInstance;
 }
 
-private static final Logger LOG = 
LoggerFactory.getLogger(StormCommon.class);
+private static final Logger LOG = getLogger(StormCommon.class);
--- End diff --

lets keep it to LoggerFactory.getLogger given that we are using the same 
way everywhere else.


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370073#comment-15370073
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194386
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
inte

[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194386
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
intervalToTaskToMetricToRegistry;
+protected final Map> streamToComponentToGrouper;
+protected final ReportErrorAndDie reportErrorDie;
+protected final Callable sampler;
+protected final AtomicBoolean backpressure;
+prot

[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370072#comment-15370072
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194309
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
inte

[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194309
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+
+public abstract class Executor implements Callable, EventHandler {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Executor.class);
+
+protected final Map workerData;
+protected final WorkerTopologyContext workerTopologyContext;
+protected final List executorId;
+protected final List taskIds;
+protected final String componentId;
+protected final AtomicBoolean openOrPrepareWasCalled;
+protected final Map stormConf;
+protected final Map conf;
+protected final String stormId;
+protected final HashMap sharedExecutorData;
+protected final AtomicBoolean stormActive;
+protected final AtomicReference> 
stormComponentDebug;
+protected final Runnable suicideFn;
+protected final IStormClusterState stormClusterState;
+protected final Map taskToComponent;
+protected CommonStats stats;
+protected final Map>> 
intervalToTaskToMetricToRegistry;
+protected final Map> streamToComponentToGrouper;
+protected final ReportErrorAndDie reportErrorDie;
+protected final Callable sampler;
+protected final AtomicBoolean backpressure;
+prot

[jira] [Commented] (STORM-1933) Intermittent test failure on test-multiple-active-storms-multiple-supervisors for supervisor-test

2016-07-10 Thread Jungtaek Lim (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1933?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370071#comment-15370071
 ] 

Jungtaek Lim commented on STORM-1933:
-

STORM-1934 is merged. I'll run the test 20 times and close this issue if 
there's no intermittent failure on 
test-multiple-active-storms-multiple-supervisors.

> Intermittent test failure on test-multiple-active-storms-multiple-supervisors 
> for supervisor-test 
> --
>
> Key: STORM-1933
> URL: https://issues.apache.org/jira/browse/STORM-1933
> Project: Apache Storm
>  Issue Type: Sub-task
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
> Attachments: 
> only-thread-1362-and-1363-BUG-60850-intermittent-failure-supervisor-test.txt
>
>
> test-multiple-active-storms-multiple-supervisors is failing with fairly high 
> chance. I've run unit test of 1.x branch 3 times and met this issue, and 
> users report FileNotFound issue on supervisor which seems to be related to 
> this.
> I have log file so I'll attach once issue is created.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370070#comment-15370070
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194012
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -539,10 +542,34 @@ protected IAuthorizer 
mkAuthorizationHandlerImpl(String klassName, Map conf) thr
 if (aznHandler != null) {
 aznHandler.prepare(conf);
 }
-LOG.debug("authorization class name:{}, class:{}, 
handler:{}",klassName, aznClass, aznHandler);
+LOG.debug("authorization class name:{}, class:{}, 
handler:{}", klassName, aznClass, aznHandler);
 }
 }
 
 return aznHandler;
 }
+
+public static WorkerTopologyContext makeWorkerContext(Map workerData) {
+try {
+StormTopology stormTopology = (StormTopology) 
workerData.get("system-topology");
--- End diff --

we should add these strings at the top with private static final String


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70194012
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/StormCommon.java ---
@@ -539,10 +542,34 @@ protected IAuthorizer 
mkAuthorizationHandlerImpl(String klassName, Map conf) thr
 if (aznHandler != null) {
 aznHandler.prepare(conf);
 }
-LOG.debug("authorization class name:{}, class:{}, 
handler:{}",klassName, aznClass, aznHandler);
+LOG.debug("authorization class name:{}, class:{}, 
handler:{}", klassName, aznClass, aznHandler);
 }
 }
 
 return aznHandler;
 }
+
+public static WorkerTopologyContext makeWorkerContext(Map workerData) {
+try {
+StormTopology stormTopology = (StormTopology) 
workerData.get("system-topology");
--- End diff --

we should add these strings at the top with private static final String


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370063#comment-15370063
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193545
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
--- End diff --

can we not do wildcard imports


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193545
  
--- Diff: storm-core/src/jvm/org/apache/storm/executor/Executor.java ---
@@ -0,0 +1,567 @@
+/**
+ * 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.executor;
+
+import clojure.lang.IFn;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.ClusterStateContext;
+import org.apache.storm.cluster.ClusterUtils;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.daemon.GrouperFactory;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.Task;
+import org.apache.storm.executor.bolt.BoltExecutor;
+import org.apache.storm.executor.error.IReportError;
+import org.apache.storm.executor.error.ReportError;
+import org.apache.storm.executor.error.ReportErrorAndDie;
+import org.apache.storm.executor.spout.SpoutExecutor;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.LoadAwareCustomStreamGrouping;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsConsumer;
+import org.apache.storm.stats.BoltExecutorStats;
+import org.apache.storm.stats.CommonStats;
+import org.apache.storm.stats.SpoutExecutorStats;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.TupleImpl;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorBackpressureCallback;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.WorkerBackpressureThread;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
--- End diff --

can we not do wildcard imports


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370062#comment-15370062
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193530
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Task.java ---
@@ -179,23 +180,22 @@ public BuiltinMetrics getBuiltInMetrics() {
 private TopologyContext mkTopologyContext(StormTopology topology) 
throws IOException {
 Map conf = (Map) workerData.get("conf");
 return new TopologyContext(
-topology,
-(Map) workerData.get("storm-conf"),
-(Map) workerData.get("task->component"),
-(Map>) 
workerData.get("component->sorted-tasks"),
-(Map>) 
workerData.get("component->stream->fields"),
-(String) workerData.get("storm-id"),
-
ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf,
 (String) workerData.get("storm-id"))),
-ConfigUtils.workerPidsRoot(conf, (String) 
workerData.get("worker-id")),
-taskId,
-(Integer) workerData.get("port"),
-(List) workerData.get("task-ids"),
-(Map) 
workerData.get("default-shared-resources"),
-(Map) workerData.get("user-shared-resources"),
-(Map) executorData.get("shared-executor-data"),
-(Map>>) 
executorData.get("interval->task->metric-registry"),
-(clojure.lang.Atom) 
executorData.get("open-or-prepare-was-called?")
-);
+topology,
+(Map) workerData.get("storm-conf"),
--- End diff --

can we move these strings into final String at the top level. I prefer if 
we add a file ExecutorDataConstants and all these there.


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193530
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Task.java ---
@@ -179,23 +180,22 @@ public BuiltinMetrics getBuiltInMetrics() {
 private TopologyContext mkTopologyContext(StormTopology topology) 
throws IOException {
 Map conf = (Map) workerData.get("conf");
 return new TopologyContext(
-topology,
-(Map) workerData.get("storm-conf"),
-(Map) workerData.get("task->component"),
-(Map>) 
workerData.get("component->sorted-tasks"),
-(Map>) 
workerData.get("component->stream->fields"),
-(String) workerData.get("storm-id"),
-
ConfigUtils.supervisorStormResourcesPath(ConfigUtils.supervisorStormDistRoot(conf,
 (String) workerData.get("storm-id"))),
-ConfigUtils.workerPidsRoot(conf, (String) 
workerData.get("worker-id")),
-taskId,
-(Integer) workerData.get("port"),
-(List) workerData.get("task-ids"),
-(Map) 
workerData.get("default-shared-resources"),
-(Map) workerData.get("user-shared-resources"),
-(Map) executorData.get("shared-executor-data"),
-(Map>>) 
executorData.get("interval->task->metric-registry"),
-(clojure.lang.Atom) 
executorData.get("open-or-prepare-was-called?")
-);
+topology,
+(Map) workerData.get("storm-conf"),
--- End diff --

can we move these strings into final String at the top level. I prefer if 
we add a file ExecutorDataConstants and all these there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1277) port backtype.storm.daemon.executor to java

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370061#comment-15370061
 ] 

ASF GitHub Bot commented on STORM-1277:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193475
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Acker.java ---
@@ -101,7 +101,7 @@ public void execute(Tuple input) {
 }
 curr.failed = true;
 pending.put(id, curr);
-} else if (ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+} else if(ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
--- End diff --

space after if


> port backtype.storm.daemon.executor to java
> ---
>
> Key: STORM-1277
> URL: https://issues.apache.org/jira/browse/STORM-1277
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-core
>Reporter: Robert Joseph Evans
>Assignee: Cody
>  Labels: java-migration, jstorm-merger
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/task
>  kind of.  Tasks and executors are combined in jstorm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1445: [STORM-1277] port backtype.storm.daemon.executor t...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1445#discussion_r70193475
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Acker.java ---
@@ -101,7 +101,7 @@ public void execute(Tuple input) {
 }
 curr.failed = true;
 pending.put(id, curr);
-} else if (ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
+} else if(ACKER_RESET_TIMEOUT_STREAM_ID.equals(streamId)) {
--- End diff --

space after if


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1883) FileReader extends Closeable Interface

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1883?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370060#comment-15370060
 ] 

ASF GitHub Bot commented on STORM-1883:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1462
  
+1 after addressing the minor comment . @roshannaik can you review as well.


> FileReader extends Closeable Interface
> --
>
> Key: STORM-1883
> URL: https://issues.apache.org/jira/browse/STORM-1883
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-hdfs
>Affects Versions: 1.0.0, 1.0.1
>Reporter: darion yaphet
>Assignee: darion yaphet
>Priority: Minor
>
> use Closeable Interface to decorate FileReader to support close()



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1462: [STORM-1883] FileReader extends Closeable Interface

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1462
  
+1 after addressing the minor comment . @roshannaik can you review as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1883) FileReader extends Closeable Interface

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1883?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370059#comment-15370059
 ] 

ASF GitHub Bot commented on STORM-1883:
---

Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1462#discussion_r70193414
  
--- Diff: 
external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileReader.java 
---
@@ -20,10 +20,11 @@
 
 import org.apache.hadoop.fs.Path;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 
-interface FileReader {
+interface FileReader extends Closeable{
--- End diff --

please add space after the Closeable


> FileReader extends Closeable Interface
> --
>
> Key: STORM-1883
> URL: https://issues.apache.org/jira/browse/STORM-1883
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-hdfs
>Affects Versions: 1.0.0, 1.0.1
>Reporter: darion yaphet
>Assignee: darion yaphet
>Priority: Minor
>
> use Closeable Interface to decorate FileReader to support close()



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1462: [STORM-1883] FileReader extends Closeable Interfac...

2016-07-10 Thread harshach
Github user harshach commented on a diff in the pull request:

https://github.com/apache/storm/pull/1462#discussion_r70193414
  
--- Diff: 
external/storm-hdfs/src/main/java/org/apache/storm/hdfs/spout/FileReader.java 
---
@@ -20,10 +20,11 @@
 
 import org.apache.hadoop.fs.Path;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 
-interface FileReader {
+interface FileReader extends Closeable{
--- End diff --

please add space after the Closeable


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1466: Update worker.xml

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1466


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1038) Upgrade netty transport from 3.x to 4.x

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370055#comment-15370055
 ] 

ASF GitHub Bot commented on STORM-1038:
---

Github user hsun-cnnxty commented on the issue:

https://github.com/apache/storm/pull/728
  
Sure.


> Upgrade netty transport from 3.x to 4.x
> ---
>
> Key: STORM-1038
> URL: https://issues.apache.org/jira/browse/STORM-1038
> Project: Apache Storm
>  Issue Type: Dependency upgrade
>  Components: storm-core
>Reporter: Hang Sun
>Priority: Minor
>  Labels: performance
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> It will be nice to upgrade netty to 4.x to take advantage of its more 
> efficient memory usage.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #728: [STORM-1038] Upgraded netty to 4.x

2016-07-10 Thread hsun-cnnxty
Github user hsun-cnnxty commented on the issue:

https://github.com/apache/storm/pull/728
  
Sure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1952) Keeping topology code for supervisor until topology got killed

2016-07-10 Thread Jungtaek Lim (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370050#comment-15370050
 ] 

Jungtaek Lim commented on STORM-1952:
-

Please correct me if I'm wrong. AFAIK, topology code is not same as cache 
because topology code is not used anymore when topology is killed. So while 
following LRU policy makes sense for this scenario, we can just remove topology 
code when supervisor knows it can be deleted.

> Keeping topology code for supervisor until topology got killed
> --
>
> Key: STORM-1952
> URL: https://issues.apache.org/jira/browse/STORM-1952
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>
> It's based on review comment from [~sriharsha].
> https://github.com/apache/storm/pull/1528/files#r69152524
> Please feel free to change reporter if you would like to.
> In supervisor we're removing topology code when assignments for that 
> supervisor has gone.
> But there's valid scenario to need to keep the topology code though 
> assignments for that supervisor is none, for example, rebalancing.
> So it would be better for supervisor to keep topology code until topology has 
> been killed (and all topology workers assigned to that supervisor are also 
> killed).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1934) Race condition between sync-supervisor and sync-processes raises several strange issues

2016-07-10 Thread Jungtaek Lim (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jungtaek Lim updated STORM-1934:

Fix Version/s: (was: 1.0.0)
   1.0.2

> Race condition between sync-supervisor and sync-processes raises several 
> strange issues
> ---
>
> Key: STORM-1934
> URL: https://issues.apache.org/jira/browse/STORM-1934
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
>Priority: Critical
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>
> There're some strange issues including STORM-1933 and others (which I will 
> file an issue soon) which are related to race condition in supervisor.
> As I mentioned to STORM-1933, basically sync-supervisor relies on zk 
> assignment, and sync-processes relies on local assignment and local workers 
> directory, but in fact sync-supervisor also access local state and take some 
> actions which affects sync-processes. And also Satish left the comment to 
> STORM-1933 describing other issue related to race condition and idea to fix 
> this which is same page on me.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370047#comment-15370047
 ] 

ASF GitHub Bot commented on STORM-1930:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1516
  
merged into 1.x-branch and master


> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
> Fix For: 2.0.0, 1.1.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1516: STORM-1930: Kafka New Client API - Support for Topic Wild...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1516
  
merged into 1.x-branch and master


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Resolved] (STORM-1922) Supervisor summary default order by host

2016-07-10 Thread Jungtaek Lim (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1922?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jungtaek Lim resolved STORM-1922.
-
   Resolution: Fixed
Fix Version/s: 1.1.0
   1.0.2
   2.0.0

> Supervisor summary default order by host
> 
>
> Key: STORM-1922
> URL: https://issues.apache.org/jira/browse/STORM-1922
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Xin Wang
>Assignee: Xin Wang
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread Jungtaek Lim (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jungtaek Lim updated STORM-1930:

Fix Version/s: (was: 1.0.2)
   1.1.0

> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
> Fix For: 2.0.0, 1.1.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1949) Backpressure can cause spout to stop emitting and stall topology

2016-07-10 Thread Jungtaek Lim (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370045#comment-15370045
 ] 

Jungtaek Lim commented on STORM-1949:
-

I'm OK to disable backpressure by default for short time, but we should work on 
resolving backpressure issue now, and enable backpressure again at next release 
or so.
This is not same as 'event logging', cause event logging is a kind of unique 
feature of Storm, but backpressure is common feature of recent streaming 
frameworks. Also backpressure is the only choice of non-ACK topology which 
needs throttle.

> Backpressure can cause spout to stop emitting and stall topology
> 
>
> Key: STORM-1949
> URL: https://issues.apache.org/jira/browse/STORM-1949
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Roshan Naik
>
> Problem can be reproduced by this [Word count 
> topology|https://github.com/hortonworks/storm/blob/perftopos1.x/examples/storm-starter/src/jvm/org/apache/storm/starter/perf/FileReadWordCountTopo.java]
> within a IDE.
> I ran it with 1 spout instance, 2 splitter bolt instances, 2 counter bolt 
> instances.
> The problem is more easily reproduced with WC topology as it causes an 
> explosion of tuples due to splitting a sentence tuple into word tuples. As 
> the bolts have to process more tuples than the  spout is producing, spout 
> needs to operate slower.
> The amount of time it takes for the topology to stall can vary.. but 
> typically under 10 mins. 
> *My theory:*  I suspect there is a race condition in the way ZK is being 
> utilized to enable/disable back pressure. When congested (i.e pressure 
> exceeds high water mark), the bolt's worker records this congested situation 
> in ZK by creating a node. Once the congestion is reduced below the low water 
> mark, it deletes this node. 
> The spout's worker has setup a watch on the parent node, expecting a callback 
> whenever there is change in the child nodes. On receiving the callback the 
> spout's worker lists the parent node to check if there are 0 or more child 
> nodes it is essentially trying to figure out the nature of state change 
> in ZK to determine whether to throttle or not. Subsequently  it setsup 
> another watch in ZK to keep an eye on future changes.
> When there are multiple bolts, there can be rapid creation/deletion of these 
> ZK nodes. Between the time the worker receives a callback and sets up the 
> next watch.. many changes may have undergone in ZK which will go unnoticed by 
> the spout. 
> The condition that the bolts are no longer congested may not get noticed as a 
> result of this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1950) Change response json of "Topology Lag" REST API to keyed by spoutId, topic, partition

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370044#comment-15370044
 ] 

ASF GitHub Bot commented on STORM-1950:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1545


> Change response json of "Topology Lag" REST API to keyed by spoutId, topic, 
> partition
> -
>
> Key: STORM-1950
> URL: https://issues.apache.org/jira/browse/STORM-1950
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-ui
>Affects Versions: 1.1.0
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
>
> From code review for STORM-1945, there's an idea to change JSON response of 
> "Topology Lag" API to keyed by topic, partition number.
> https://github.com/apache/storm/pull/1541#issuecomment-230983140
> I think also make result keyed by spout id would be good.
> Here's sample JSON of output after this issue is resolved.
> {code}
> {
>"spout1":{
>   "spoutId":"spout1",
>   "spoutType":"KAFKA",
>   "spoutLagResult":{
>  "topic":{
> "partition0":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> },
> "partition2":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> }
>  },
>  "topic2":{
> "partition0":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> },
> "partition2":{
>"consumerCommittedOffset":1175610,
>"logHeadOffset":5634192,
>"lag":4458582
> }
>  }
>   }
>}
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1545: STORM-1950 Change response json of "Topology Lag" ...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1545


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1737) storm-kafka-client has compilation errors with Apache Kafka 0.10

2016-07-10 Thread Jungtaek Lim (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370040#comment-15370040
 ] 

Jungtaek Lim commented on STORM-1737:
-

If this issue can be fixed by several days I'm OK to mark this as 'Blocker', 
but if not I don't think this issue should block releasing 1.0.2.
Yes Kafka is a kind of de-facto standard queue now so we should consider Kafka 
as first class, but Apache Kafka 0.10 was released just 1 month ago and no 
other streaming frameworks seem support Kafka 0.10.

Btw, can we make storm-kafka-client not using methods marked as 'Unstable'? 
Since it's marked as 'Unstable' Kafka community can change that at other minor 
updates, and I don't think we should follow their changes ASAP.

> storm-kafka-client has compilation errors with Apache Kafka 0.10
> 
>
> Key: STORM-1737
> URL: https://issues.apache.org/jira/browse/STORM-1737
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Sriharsha Chintalapani
>Assignee: Hugo Louro
>Priority: Blocker
> Fix For: 1.0.2
>
>
> when compiled with Apache Kafka 0.10 branch getting following errors
> {code}
> [ERROR] 
> /Users/harsha/code/hwx/storm/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java:[163,51]
>  incompatible types: org.apache.kafka.common.TopicPartition cannot be 
> converted to java.util.Collection
> [ERROR] 
> /Users/harsha/code/hwx/storm/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java:[166,45]
>  incompatible types: org.apache.kafka.common.TopicPartition cannot be 
> converted to java.util.Collection
> [ERROR] 
> /Users/harsha/code/hwx/storm/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java:[175,51]
>  incompatible types: org.apache.kafka.common.TopicPartition cannot be 
> converted to java.util.Collection
> [ERROR] 
> /Users/harsha/code/hwx/storm/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java:[177,45]
>  incompatible types: org.apache.kafka.common.TopicPartition cannot be 
> converted to java.util.Collection
> [ERROR] 
> /Users/harsha/code/hwx/storm/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java:[252,41]
>  incompatible types: org.apache.kafka.common.TopicPartition cannot be 
> converted to java.util.Collection
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1038) Upgrade netty transport from 3.x to 4.x

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370036#comment-15370036
 ] 

ASF GitHub Bot commented on STORM-1038:
---

Github user HeartSaVioR commented on the issue:

https://github.com/apache/storm/pull/728
  
After rebasing, could you do the performance test against 1.x branch? The 
status of master branch is a  WIP so we would be more convenient with 1.x 
branch.


> Upgrade netty transport from 3.x to 4.x
> ---
>
> Key: STORM-1038
> URL: https://issues.apache.org/jira/browse/STORM-1038
> Project: Apache Storm
>  Issue Type: Dependency upgrade
>  Components: storm-core
>Reporter: Hang Sun
>Priority: Minor
>  Labels: performance
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> It will be nice to upgrade netty to 4.x to take advantage of its more 
> efficient memory usage.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #728: [STORM-1038] Upgraded netty to 4.x

2016-07-10 Thread HeartSaVioR
Github user HeartSaVioR commented on the issue:

https://github.com/apache/storm/pull/728
  
After rebasing, could you do the performance test against 1.x branch? The 
status of master branch is a  WIP so we would be more convenient with 1.x 
branch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1038) Upgrade netty transport from 3.x to 4.x

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15370025#comment-15370025
 ] 

ASF GitHub Bot commented on STORM-1038:
---

Github user hsun-cnnxty commented on the issue:

https://github.com/apache/storm/pull/728
  
I am currently on vacation and will be back in two weeks.  Will work on it 
as soon as I am back home.

-thanks


> Upgrade netty transport from 3.x to 4.x
> ---
>
> Key: STORM-1038
> URL: https://issues.apache.org/jira/browse/STORM-1038
> Project: Apache Storm
>  Issue Type: Dependency upgrade
>  Components: storm-core
>Reporter: Hang Sun
>Priority: Minor
>  Labels: performance
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> It will be nice to upgrade netty to 4.x to take advantage of its more 
> efficient memory usage.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #728: [STORM-1038] Upgraded netty to 4.x

2016-07-10 Thread hsun-cnnxty
Github user hsun-cnnxty commented on the issue:

https://github.com/apache/storm/pull/728
  
I am currently on vacation and will be back in two weeks.  Will work on it 
as soon as I am back home.

-thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1038) Upgrade netty transport from 3.x to 4.x

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369969#comment-15369969
 ] 

ASF GitHub Bot commented on STORM-1038:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/728
  
@hsun-cnnxty we would like to get this into 1.x-branch as well as master. 
Did you get a chance to look at @revans2 comment above . It will be great if 
you can address the comment and up merge your patch.


> Upgrade netty transport from 3.x to 4.x
> ---
>
> Key: STORM-1038
> URL: https://issues.apache.org/jira/browse/STORM-1038
> Project: Apache Storm
>  Issue Type: Dependency upgrade
>  Components: storm-core
>Reporter: Hang Sun
>Priority: Minor
>  Labels: performance
>   Original Estimate: 168h
>  Remaining Estimate: 168h
>
> It will be nice to upgrade netty to 4.x to take advantage of its more 
> efficient memory usage.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1129) Storm should use topology name instead of ids for url in storm UI.

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1129?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369970#comment-15369970
 ] 

ASF GitHub Bot commented on STORM-1129:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1277
  
@priyank5485 sorry for the delay on this. Can you up merge this patch.


> Storm should use topology name instead of ids for url in storm UI.
> --
>
> Key: STORM-1129
> URL: https://issues.apache.org/jira/browse/STORM-1129
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Reporter: Priyank Shah
>Assignee: Priyank Shah
>
> Currently, in storm UI details about a topology can be viewed at a URL which 
> has a topology id as a query parameter. When a topology is updated and  
> redeployed a new id is assigned by storm and existing URL(and any bookmarks 
> relying on it) for the topology do not work since the id has changed. We 
> should change it so that topology name is used instead of id.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1277: STORM-1129: Update ui to use topology name

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1277
  
@priyank5485 sorry for the delay on this. Can you up merge this patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #728: [STORM-1038] Upgraded netty to 4.x

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/728
  
@hsun-cnnxty we would like to get this into 1.x-branch as well as master. 
Did you get a chance to look at @revans2 comment above . It will be great if 
you can address the comment and up merge your patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-720) Storm.cmd should return ERRORLEVEL before exiting main block

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369965#comment-15369965
 ] 

ASF GitHub Bot commented on STORM-720:
--

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/476
  
@knusbaum are we good to merge this?


> Storm.cmd should return ERRORLEVEL before exiting main block
> 
>
> Key: STORM-720
> URL: https://issues.apache.org/jira/browse/STORM-720
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 0.9.3, 0.9.4
>Reporter: Ravi Tandon
>Assignee: Ravi Tandon
>Priority: Trivial
> Fix For: 0.9.3, 0.9.4
>
> Attachments: storm_cmd_exitcode_rev0.patch
>
>
> This JIRA is for a very small PR that I will post soon (attached patch)
> Issue: The Storm.cmd does not exit with an ErrorLevel.
> Impact: Any automation via windows banks on return or exit code from the 
> program to determine success.
> When can this occur: Passing wrong arguments (like wrong class name or bad 
> topology builder code) to java will result in error and any automation 
> program will check on the exit code.
> FYI @ [~harsha_ch] [~shanyu]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #476: STORM-720: Storm.cmd should return ERRORLEVEL before exiti...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/476
  
@knusbaum are we good to merge this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1833) Add simple equi-join support in storm-sql standalone mode

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369964#comment-15369964
 ] 

ASF GitHub Bot commented on STORM-1833:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1413
  
Thanks @arunmahadevan merged into master, 1.x-branch


> Add simple equi-join support in storm-sql standalone mode
> -
>
> Key: STORM-1833
> URL: https://issues.apache.org/jira/browse/STORM-1833
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Arun Mahadevan
>Assignee: Arun Mahadevan
>
> Provide simple equi join support in storm sql standalone mode.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1413: [STORM-1833] Simple equi-join in storm-sql standalone mod...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1413
  
Thanks @arunmahadevan merged into master, 1.x-branch


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1833) Add simple equi-join support in storm-sql standalone mode

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369963#comment-15369963
 ] 

ASF GitHub Bot commented on STORM-1833:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1413


> Add simple equi-join support in storm-sql standalone mode
> -
>
> Key: STORM-1833
> URL: https://issues.apache.org/jira/browse/STORM-1833
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Arun Mahadevan
>Assignee: Arun Mahadevan
>
> Provide simple equi join support in storm sql standalone mode.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1413: [STORM-1833] Simple equi-join in storm-sql standal...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1413


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1833) Add simple equi-join support in storm-sql standalone mode

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369955#comment-15369955
 ] 

ASF GitHub Bot commented on STORM-1833:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1413
  
+1


> Add simple equi-join support in storm-sql standalone mode
> -
>
> Key: STORM-1833
> URL: https://issues.apache.org/jira/browse/STORM-1833
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Arun Mahadevan
>Assignee: Arun Mahadevan
>
> Provide simple equi join support in storm sql standalone mode.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1413: [STORM-1833] Simple equi-join in storm-sql standalone mod...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1413
  
+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1674) Idle KafkaSpout consumes more bandwidth than needed

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369954#comment-15369954
 ] 

ASF GitHub Bot commented on STORM-1674:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1287
  
@moesol can you please close this PR . Its already merged here 
https://github.com/apache/storm/commit/f069cfb08099a54448bba3f96f20c5b929ea6af3


> Idle KafkaSpout consumes more bandwidth than needed
> ---
>
> Key: STORM-1674
> URL: https://issues.apache.org/jira/browse/STORM-1674
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-kafka
>Affects Versions: 0.9.3, 0.10.1, 1.0.1
>Reporter: Robert Hastings
>Assignee: Robert Hastings
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>
> Discovered 30 megabits of traffic flowing between a set of KafkaSpouts
> and our kafka servers even though no Kafka messages were moving.
> Using the wireshark kafka dissector, we were able to see that
> each FetchRequest had maxWait set to 1
> and minBytes set to 0. When binBytes is set to 0 the kafka server
> responds immediately when there are no messages. In turn the KafkaSpout
> polls without any delay causing a constant stream of FetchRequest/
> FetchResponse messages. Using a non-KafkaSpout client had a similar
> traffic pattern with two key differences
> 1) minBytes was 1
> 2) maxWait was 100
> With these FetchRequest parameters and no messages flowing,
> the kafka server delays the FetchResponse by 100 ms. This reduces
> the network traffic from megabits to the low kilobits. It also
> reduced the CPU utilization of our kafka server from 140% to 2%.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1287: STORM-1674: Addresses network flood from KafkaSpout to ka...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1287
  
@moesol can you please close this PR . Its already merged here 
https://github.com/apache/storm/commit/f069cfb08099a54448bba3f96f20c5b929ea6af3


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1885) python script for squashing and merging prs

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1885?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369952#comment-15369952
 ] 

ASF GitHub Bot commented on STORM-1885:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1468
  
@HeartSaVioR Thanks for documenting the script.
"Branch policy is not compatible with projects which uses this script. They 
have branches per version but we just maintain version lines (major, minor, 
bugfix) so we should do something while determining fix versions from merged 
branches."
Agree but than again we need modify either script to get this done . So 
spark or kafka script doesn't matter much.
"We're having master and 1.x / 1.0.x branches heavily diverged, so there're 
often two or more pull requests submitted per one issue. (Committers don't 
cherry-pick between master and 1.x for storm-core manually since it's easy to 
see merge conflict.) It should be tested (at least unit test and integration 
test) individually, and issue should be closed when all of pull requests are 
checked in. It means that we're having different merging step which other 
projects don't have.
Moreover, commit message hook (closing PR) doesn't work if PR is not 
against master."
I am +1 on adding unit tests, integration tests to be run as part of the 
script but have a manual validation from the user to say yes/no to go-ahead 
with merge. As the unit tests can be flaky there can be false negatives.
"We should update CHANGELOG while merging step. Personally I don't like 
updating CHANGELOG so opened thread for discussion but we didn't decide 
something clearly."
Agree with you on this. This extra-step adds unnecessary commits to the 
log. As long as we update the JIRA fixVersions before the release its easy to 
generate a change log.

"Commit message will contain body of pull request which is free format for 
now and tends to be meaningless for commit message."
With the above script one can edit the commit title and it will have JIRA 
number & title that will give us much more meaningful message about the work 
done as part of th ejira.

"So without arranging our branch policy and merging step, it will be hard 
to get merge script fit for us."
I disagree with this. We can have minimal modifications to the script to 
work with our current branches.




> python script for squashing and merging prs
> ---
>
> Key: STORM-1885
> URL: https://issues.apache.org/jira/browse/STORM-1885
> Project: Apache Storm
>  Issue Type: Task
>Reporter: Sriharsha Chintalapani
>Assignee: Sriharsha Chintalapani
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1468: STORM-1885. python script for squashing and merging prs.

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1468
  
@HeartSaVioR Thanks for documenting the script.
"Branch policy is not compatible with projects which uses this script. They 
have branches per version but we just maintain version lines (major, minor, 
bugfix) so we should do something while determining fix versions from merged 
branches."
Agree but than again we need modify either script to get this done . So 
spark or kafka script doesn't matter much.
"We're having master and 1.x / 1.0.x branches heavily diverged, so there're 
often two or more pull requests submitted per one issue. (Committers don't 
cherry-pick between master and 1.x for storm-core manually since it's easy to 
see merge conflict.) It should be tested (at least unit test and integration 
test) individually, and issue should be closed when all of pull requests are 
checked in. It means that we're having different merging step which other 
projects don't have.
Moreover, commit message hook (closing PR) doesn't work if PR is not 
against master."
I am +1 on adding unit tests, integration tests to be run as part of the 
script but have a manual validation from the user to say yes/no to go-ahead 
with merge. As the unit tests can be flaky there can be false negatives.
"We should update CHANGELOG while merging step. Personally I don't like 
updating CHANGELOG so opened thread for discussion but we didn't decide 
something clearly."
Agree with you on this. This extra-step adds unnecessary commits to the 
log. As long as we update the JIRA fixVersions before the release its easy to 
generate a change log.

"Commit message will contain body of pull request which is free format for 
now and tends to be meaningless for commit message."
With the above script one can edit the commit title and it will have JIRA 
number & title that will give us much more meaningful message about the work 
done as part of th ejira.

"So without arranging our branch policy and merging step, it will be hard 
to get merge script fit for us."
I disagree with this. We can have minimal modifications to the script to 
work with our current branches.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1927) Upgrade Jetty and Ring

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1927?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369949#comment-15369949
 ] 

ASF GitHub Bot commented on STORM-1927:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1520
  
+1


> Upgrade Jetty and Ring
> --
>
> Key: STORM-1927
> URL: https://issues.apache.org/jira/browse/STORM-1927
> Project: Apache Storm
>  Issue Type: Improvement
>  Components: storm-ui
>Affects Versions: 1.0.0, 1.0.1
>Reporter: darion yaphet
>Assignee: darion yaphet
>
> Jetty 7 is EOL , upgrade to Jetty 9 & Ring could also support it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1520: [STORM-1927] Upgrade Jetty and Ring

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1520
  
+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1866) Update Resource Aware Scheduler Documentation

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369948#comment-15369948
 ] 

ASF GitHub Bot commented on STORM-1866:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1510


> Update Resource Aware Scheduler Documentation
> -
>
> Key: STORM-1866
> URL: https://issues.apache.org/jira/browse/STORM-1866
> Project: Apache Storm
>  Issue Type: Story
>Reporter: Boyang Jerry Peng
>Assignee: Boyang Jerry Peng
>Priority: Trivial
>
> The new features for RAS (i.e. STORM-1766) needs to be documented and there 
> are a couple links in the RAS documentation that does not work anymore



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1510: [STORM-1866] - Update Resource Aware Scheduler Doc...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1510


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Updated] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated STORM-1930:
--
Fix Version/s: (was: 1.1.0)

> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
> Fix For: 2.0.0, 1.0.2
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani resolved STORM-1930.
---
   Resolution: Fixed
Fix Version/s: 1.1.0
   1.0.2
   2.0.0

> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1942) Extra closing div tag in topology.html

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1942?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated STORM-1942:
--
Fix Version/s: 1.1.0
   2.0.0

> Extra closing div tag in topology.html
> --
>
> Key: STORM-1942
> URL: https://issues.apache.org/jira/browse/STORM-1942
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-ui
>Affects Versions: 2.0.0
>Reporter: Alessandro Bellina
>Assignee: Alessandro Bellina
>Priority: Minor
> Fix For: 2.0.0, 1.1.0
>
>
> Extra  in topology.html causing styling to be strage. Appears to have 
> been introduced in STORM-1136.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (STORM-1942) Extra closing div tag in topology.html

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1942?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani resolved STORM-1942.
---
Resolution: Fixed

> Extra closing div tag in topology.html
> --
>
> Key: STORM-1942
> URL: https://issues.apache.org/jira/browse/STORM-1942
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-ui
>Affects Versions: 2.0.0
>Reporter: Alessandro Bellina
>Assignee: Alessandro Bellina
>Priority: Minor
> Fix For: 2.0.0, 1.1.0
>
>
> Extra  in topology.html causing styling to be strage. Appears to have 
> been introduced in STORM-1136.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1942) Extra closing div tag in topology.html

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369946#comment-15369946
 ] 

ASF GitHub Bot commented on STORM-1942:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1537


> Extra closing div tag in topology.html
> --
>
> Key: STORM-1942
> URL: https://issues.apache.org/jira/browse/STORM-1942
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-ui
>Affects Versions: 2.0.0
>Reporter: Alessandro Bellina
>Assignee: Alessandro Bellina
>Priority: Minor
> Fix For: 2.0.0, 1.1.0
>
>
> Extra  in topology.html causing styling to be strage. Appears to have 
> been introduced in STORM-1136.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1537: STORM-1942: remove extra div that was causing weir...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1537


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1922) Supervisor summary default order by host

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1922?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369945#comment-15369945
 ] 

ASF GitHub Bot commented on STORM-1922:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1519
  
Thanks @vesense merged into master,1.x, 1.0.x


> Supervisor summary default order by host
> 
>
> Key: STORM-1922
> URL: https://issues.apache.org/jira/browse/STORM-1922
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Xin Wang
>Assignee: Xin Wang
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1519: STORM-1922: Supervisor summary default order by host

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1519
  
Thanks @vesense merged into master,1.x, 1.0.x


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1922) Supervisor summary default order by host

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1922?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369944#comment-15369944
 ] 

ASF GitHub Bot commented on STORM-1922:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1519


> Supervisor summary default order by host
> 
>
> Key: STORM-1922
> URL: https://issues.apache.org/jira/browse/STORM-1922
> Project: Apache Storm
>  Issue Type: Improvement
>Reporter: Xin Wang
>Assignee: Xin Wang
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1519: STORM-1922: Supervisor summary default order by ho...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1519


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1866) Update Resource Aware Scheduler Documentation

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1866?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369943#comment-15369943
 ] 

ASF GitHub Bot commented on STORM-1866:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1510
  
+1


> Update Resource Aware Scheduler Documentation
> -
>
> Key: STORM-1866
> URL: https://issues.apache.org/jira/browse/STORM-1866
> Project: Apache Storm
>  Issue Type: Story
>Reporter: Boyang Jerry Peng
>Assignee: Boyang Jerry Peng
>Priority: Trivial
>
> The new features for RAS (i.e. STORM-1766) needs to be documented and there 
> are a couple links in the RAS documentation that does not work anymore



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1510: [STORM-1866] - Update Resource Aware Scheduler Documentat...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1510
  
+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1516: STORM-1930: Kafka New Client API - Support for Topic Wild...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1516
  
+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369941#comment-15369941
 ] 

ASF GitHub Bot commented on STORM-1930:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1516
  
+1


> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1930) Kafka New Client API - Support for Topic Wildcards

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369940#comment-15369940
 ] 

ASF GitHub Bot commented on STORM-1930:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1516


> Kafka New Client API - Support for Topic Wildcards
> --
>
> Key: STORM-1930
> URL: https://issues.apache.org/jira/browse/STORM-1930
> Project: Apache Storm
>  Issue Type: New Feature
>  Components: storm-kafka
>Affects Versions: 1.0.2
>Reporter: Hugo Louro
>Assignee: Hugo Louro
>Priority: Critical
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm pull request #1516: STORM-1930: Kafka New Client API - Support for Top...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1516


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1546: Storm 1924: Adding conf options for Persistent Word Count...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1546
  
Thanks @csivaguru merged into 1.x-branch and master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1546: Storm 1924: Adding conf options for Persistent Wor...

2016-07-10 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1546


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (STORM-1954) Large Trident topologies can cause memory issues due to DefaultResourceDeclarer object reading config

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1954?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369931#comment-15369931
 ] 

ASF GitHub Bot commented on STORM-1954:
---

Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1547
  
+1


> Large Trident topologies can cause memory issues due to 
> DefaultResourceDeclarer object reading config
> -
>
> Key: STORM-1954
> URL: https://issues.apache.org/jira/browse/STORM-1954
> Project: Apache Storm
>  Issue Type: Bug
>Reporter: Kyle Nusbaum
>
> DefaultResourceDeclarer objects each read the full config on instantiation. 
> We've seen this causing slow startup and OOMs for large Trident topologies. 
> The config should be static in DefaultResourceDeclarer.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] storm issue #1547: STORM-1954: Large Trident topologies can cause memory iss...

2016-07-10 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1547
  
+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Updated] (STORM-1934) Race condition between sync-supervisor and sync-processes raises several strange issues

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated STORM-1934:
--
Fix Version/s: 1.1.0
   2.0.0
   1.0.0

> Race condition between sync-supervisor and sync-processes raises several 
> strange issues
> ---
>
> Key: STORM-1934
> URL: https://issues.apache.org/jira/browse/STORM-1934
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
>Priority: Critical
> Fix For: 1.0.0, 2.0.0, 1.1.0
>
>
> There're some strange issues including STORM-1933 and others (which I will 
> file an issue soon) which are related to race condition in supervisor.
> As I mentioned to STORM-1933, basically sync-supervisor relies on zk 
> assignment, and sync-processes relies on local assignment and local workers 
> directory, but in fact sync-supervisor also access local state and take some 
> actions which affects sync-processes. And also Satish left the comment to 
> STORM-1933 describing other issue related to race condition and idea to fix 
> this which is same page on me.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (STORM-1956) Disable Backpressure by default

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani resolved STORM-1956.
---
Resolution: Fixed

> Disable Backpressure by default
> ---
>
> Key: STORM-1956
> URL: https://issues.apache.org/jira/browse/STORM-1956
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 1.0.1
>Reporter: Roshan Naik
>Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>
> Some of the context on this is captured in STORM-1949 
> In short.. wait for BP mechanism to mature some more and be production ready 
> before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (STORM-1934) Race condition between sync-supervisor and sync-processes raises several strange issues

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1934?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani resolved STORM-1934.
---
Resolution: Fixed

> Race condition between sync-supervisor and sync-processes raises several 
> strange issues
> ---
>
> Key: STORM-1934
> URL: https://issues.apache.org/jira/browse/STORM-1934
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 2.0.0, 1.0.1
>Reporter: Jungtaek Lim
>Assignee: Jungtaek Lim
>Priority: Critical
> Fix For: 1.0.0, 2.0.0, 1.1.0
>
>
> There're some strange issues including STORM-1933 and others (which I will 
> file an issue soon) which are related to race condition in supervisor.
> As I mentioned to STORM-1933, basically sync-supervisor relies on zk 
> assignment, and sync-processes relies on local assignment and local workers 
> directory, but in fact sync-supervisor also access local state and take some 
> actions which affects sync-processes. And also Satish left the comment to 
> STORM-1933 describing other issue related to race condition and idea to fix 
> this which is same page on me.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (STORM-1956) Disable Backpressure by default

2016-07-10 Thread Sriharsha Chintalapani (JIRA)

 [ 
https://issues.apache.org/jira/browse/STORM-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sriharsha Chintalapani updated STORM-1956:
--
Fix Version/s: 1.1.0
   2.0.0

> Disable Backpressure by default
> ---
>
> Key: STORM-1956
> URL: https://issues.apache.org/jira/browse/STORM-1956
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 1.0.1
>Reporter: Roshan Naik
>Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 2.0.0, 1.0.2, 1.1.0
>
>
> Some of the context on this is captured in STORM-1949 
> In short.. wait for BP mechanism to mature some more and be production ready 
> before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (STORM-1956) Disable Backpressure by default

2016-07-10 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/STORM-1956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15369926#comment-15369926
 ] 

ASF GitHub Bot commented on STORM-1956:
---

Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1548


> Disable Backpressure by default
> ---
>
> Key: STORM-1956
> URL: https://issues.apache.org/jira/browse/STORM-1956
> Project: Apache Storm
>  Issue Type: Bug
>  Components: storm-core
>Affects Versions: 1.0.0, 1.0.1
>Reporter: Roshan Naik
>Assignee: Roshan Naik
>Priority: Blocker
> Fix For: 1.0.2
>
>
> Some of the context on this is captured in STORM-1949 
> In short.. wait for BP mechanism to mature some more and be production ready 
> before we enable by default.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


  1   2   >