[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-15 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-14 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87902202
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.
+

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87187329
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87191222
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87336388
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/worker/WorkerState.java ---
@@ -0,0 +1,690 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.StormTimer;
+import org.apache.storm.cluster.IStateStorage;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.cluster.VersionedData;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.supervisor.AdvancedFSOps;
+import org.apache.storm.generated.Assignment;
+import org.apache.storm.generated.DebugOptions;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.NodeInfo;
+import org.apache.storm.generated.StormBase;
+import org.apache.storm.generated.StormBase;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.StreamInfo;
+import org.apache.storm.generated.TopologyStatus;
+import org.apache.storm.grouping.Load;
+import org.apache.storm.grouping.LoadMapping;
+import org.apache.storm.hooks.BaseWorkerHook;
+import org.apache.storm.messaging.ConnectionWithStatus;
+import org.apache.storm.messaging.DeserializingConnectionCallback;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.messaging.TransportFactory;
+import org.apache.storm.serialization.KryoTupleSerializer;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.tuple.AddressedTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DisruptorQueue;
+import org.apache.storm.utils.ThriftTopologyUtils;
+import org.apache.storm.utils.TransferDrainer;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
+import java.util.function.UnaryOperator;
+import java.util.stream.Collectors;
+
+public class WorkerState {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(WorkerState.class);
+
+final Map conf;
+final IContext mqContext;
+
+public Map getConf() {
+return conf;
+}
+
+public IConnection getReceiver() {
+return receiver;
+}
+
+public String getTopologyId() {
+return topologyId;
+}
+
+public int getPort() {
+return port;
+}
+
+public String getWorkerId() {
+return workerId;
+}
+
+public IStateStorage getStateStorage() {
+return stateStorage;
+}
+
+public AtomicBoolean getIsTopologyActive() {
+return isTopologyActive;
+}
+
+public AtomicReference> 
getStormComponentToDebug() {
+return stormComponentToDebug;
+}
+
+public Set getExecutors() {
+return executors;
+}
+
+public List getTaskIds() {
+return taskIds;
+}
+
+public Map getTopologyConf() {
+return topologyConf;
  

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87337480
  
--- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
@@ -123,6 +123,7 @@
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicReference;
--- End diff --

Nit: new import without actual usage.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87191290
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87160199
  
--- Diff: storm-core/src/jvm/org/apache/storm/Constants.java ---
@@ -20,12 +20,15 @@
 import org.apache.storm.coordination.CoordinatedBolt;
 import clojure.lang.RT;
--- End diff --

Nitpick: We could remove RT now.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87189395
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87163440
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
--- End diff --

Minor or maybe nitpick: it would be better to expand here and below three.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87161787
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java ---
@@ -0,0 +1,157 @@
+/**
+ * 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.daemon.worker;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.storm.utils.Time;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.LogLevel;
+import org.apache.storm.generated.LogLevelAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class LogConfigManager {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(LogConfigManager.class);
+
+private final AtomicReference> 
latestLogConfig;
+private final Map originalLogLevels;
+
+public LogConfigManager() {
+this(new AtomicReference<>(new TreeMap<>()));
+}
+
+public LogConfigManager(AtomicReference> 
latestLogConfig) {
+this.latestLogConfig = latestLogConfig;
+this.originalLogLevels = getLoggerLevels();
+LOG.info("Started with log levels: {}", originalLogLevels);
+}
+
+public void processLogConfigChange(LogConfig logConfig) {
+if (null != logConfig) {
+LOG.debug("Processing received log config: {}", logConfig);
+TreeMap loggers = new 
TreeMap<>(logConfig.get_named_logger_level());
+LoggerContext logContext = (LoggerContext) 
LogManager.getContext(false);
+Map newLogConfigs = new HashMap<>();
+for (Map.Entry entry : loggers.entrySet()) {
+String msgLoggerName = entry.getKey();
+msgLoggerName = ("ROOT".equalsIgnoreCase(msgLoggerName)) ? 
LogManager.ROOT_LOGGER_NAME : msgLoggerName;
+LogLevel loggerLevel = entry.getValue();
+// the new-timeouts map now contains logger => timeout
+if (loggerLevel.is_set_reset_log_level_timeout_epoch()) {
+LogLevel copy = new LogLevel(loggerLevel);
+if (originalLogLevels.containsKey(msgLoggerName)) {
+
copy.set_reset_log_level(originalLogLevels.get(msgLoggerName).name());
+} else {
+copy.set_reset_log_level(Level.INFO.name());
+}
+
+//copy.unset_reset_log_level();
--- End diff --

Nitpick: remove comment here.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-09 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r87190721
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-03 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86344854
  
--- Diff: storm-core/src/clj/org/apache/storm/testing.clj ---
@@ -667,23 +665,18 @@
   (.put "spout-emitted" (AtomicInteger. 0))
   (.put "transferred" (AtomicInteger. 0))
   (.put "processed" (AtomicInteger. 0
+  (LocalExecutor/setTrackId id#)
   (with-var-roots
--- End diff --

We don't need with-var-roots any more, we could just call with 
simulated-time-local-cluster without it


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-02 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86233780
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/Task.java ---
@@ -179,22 +178,21 @@ public BuiltinMetrics getBuiltInMetrics() {
 }
 
 private TopologyContext mkTopologyContext(StormTopology topology) 
throws IOException {
-Map conf = (Map) workerData.get(Constants.CONF);
+Map conf = workerData.getConf();
 return new TopologyContext(
-topology,
-(Map) workerData.get(Constants.STORM_CONF),
-(Map) 
workerData.get(Constants.TASK_TO_COMPONENT),
-(Map) 
workerData.get(Constants.COMPONENT_TO_SORTED_TASKS),
-(Map>) 
workerData.get(Constants.COMPONENT_TO_STREAM_TO_FIELDS),
-(String) workerData.get(Constants.STORM_ID),
+topology,
+workerData.getTopologyConf(),
+workerData.getTaskToComponent(),
+workerData.getComponentToSortedTasks(),
+workerData.getComponentToStreamToFields(),
+workerData.getTopologyId(),
 ConfigUtils.supervisorStormResourcesPath(
-ConfigUtils.supervisorStormDistRoot(conf, (String) 
workerData.get(Constants.STORM_ID))),
-ConfigUtils.workerPidsRoot(conf, (String) 
workerData.get(Constants.WORKER_ID)),
+ConfigUtils.supervisorStormDistRoot(conf, 
workerData.getTopologyId())),
+ConfigUtils.workerPidsRoot(conf, workerData.getWorkerId()),
 taskId,
-(Integer) workerData.get(Constants.PORT),
-(List) workerData.get(Constants.TASK_IDS),
-(Map) 
workerData.get(Constants.DEFAULT_SHARED_RESOURCES),
-(Map) 
workerData.get(Constants.USER_SHARED_RESOURCES),
+workerData.getPort(), workerData.getTaskIds(),
+workerData.getDefaultSharedResources(),
+   workerData.getUserSharedResources(),
--- End diff --

nit: indentation is off


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-02 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86233431
  
--- Diff: storm-core/src/clj/org/apache/storm/daemon/local_executor.clj ---
@@ -25,18 +25,3 @@
 (let [val (AddressedTuple. task tuple)]
--- End diff --

This entire file should really be removed and with-tracked-cluster should 
be updated to not use it.  It is obviously not needed because this function is 
never called, only over ridden.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-02 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86168341
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.
+

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-02 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86166475
  
--- Diff: storm-core/src/jvm/org/apache/storm/daemon/worker/Worker.java ---
@@ -0,0 +1,426 @@
+/**
+ * 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.daemon.worker;
+
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.ObjectUtils;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.*;
+import org.apache.storm.daemon.DaemonCommon;
+import org.apache.storm.daemon.Shutdownable;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.executor.Executor;
+import org.apache.storm.executor.ExecutorShutdown;
+import org.apache.storm.executor.IRunningExecutor;
+import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.*;
+import org.apache.storm.messaging.IConnection;
+import org.apache.storm.messaging.IContext;
+import org.apache.storm.messaging.TaskMessage;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.*;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.security.PrivilegedExceptionAction;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class Worker implements Shutdownable, DaemonCommon {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(Worker.class);
+private final Map conf;
+private final IContext context;
+private final String topologyId;
+private final String assignmentId;
+private final int port;
+private final String workerId;
+private final LogConfigManager logConfigManager;
+
+
+private WorkerState workerState;
+private AtomicReference executorsAtom;
+private Thread transferThread;
+private WorkerBackpressureThread backpressureThread;
+
+private AtomicReference credentialsAtom;
+private Subject subject;
+private Collection autoCreds;
+
+
+/**
+ * TODO: should worker even take the topologyId as input? this should 
be
+ * deducable from cluster state (by searching through assignments)
+ * what about if there's inconsistency in assignments? -> but nimbus 
should guarantee this consistency
+ *
+ * @param conf - Storm configuration
+ * @param context  -
+ * @param topologyId   - topology id
+ * @param assignmentId - assignement id
+ * @param port - port on which the worker runs
+ * @param workerId - worker id
+ */
+
+public Worker(Map conf, IContext context, String topologyId, String 
assignmentId, int port, String workerId) {
+this.conf = conf;
+this.context = context;
+this.topologyId = topologyId;
+this.assignmentId = assignmentId;
+this.port = port;
+this.workerId = workerId;
+this.logConfigManager = new LogConfigManager();
+}
+
+public void start() throws Exception {
+LOG.info("Launching worker for {} on {}:{} with id {} and conf 
{}", topologyId, assignmentId, port, workerId,
+conf);
+// because in local mode, its not a separate
+// process. supervisor will register it in this case
+// if ConfigUtils.isLocalMode(conf) returns false then it is in 
distributed mode.
+

[GitHub] storm pull request #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

2016-11-02 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1756#discussion_r86178116
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/worker/LogConfigManager.java ---
@@ -0,0 +1,154 @@
+/**
+ * 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.daemon.worker;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.LogLevel;
+import org.apache.storm.generated.LogLevelAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class LogConfigManager {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(LogConfigManager.class);
+
+private final AtomicReference> 
latestLogConfig;
+private final Map originalLogLevels;
+
+public LogConfigManager() {
+this(new AtomicReference<>(new TreeMap<>()));
+}
+
+public LogConfigManager(AtomicReference> 
latestLogConfig) {
+this.latestLogConfig = latestLogConfig;
+this.originalLogLevels = getLoggerLevels();
+LOG.info("Started with log levels: {}", originalLogLevels);
+}
+
+public void processLogConfigChange(LogConfig logConfig) {
+if (null != logConfig) {
+LOG.debug("Processing received log config: {}", logConfig);
+TreeMap loggers = new 
TreeMap<>(logConfig.get_named_logger_level());
+LoggerContext logContext = (LoggerContext) 
LogManager.getContext();
--- End diff --

In the previous code it passed in a false.  Not sure if it makes much 
difference in our case though.


https://logging.apache.org/log4j/2.x/log4j-api/apidocs/org/apache/logging/log4j/LogManager.html#getContext(boolean)


---
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.
---