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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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<List<IRunningExecutor>> executorsAtom;
    +    private Thread transferThread;
    +    private WorkerBackpressureThread backpressureThread;
    +
    +    private AtomicReference<Credentials> credentialsAtom;
    +    private Subject subject;
    +    private Collection<IAutoCredentials> 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.
    +        if (!ConfigUtils.isLocalMode(conf)) {
    +            // Distributed mode
    +            SysOutOverSLF4J.sendSystemOutAndErrToSLF4J();
    +            String pid = Utils.processPid();
    +            FileUtils.touch(new File(ConfigUtils.workerPidPath(conf, 
workerId, pid)));
    +            FileUtils.writeStringToFile(new 
File(ConfigUtils.workerArtifactsPidPath(conf, topologyId, port)), pid,
    +                Charset.forName("UTF-8"));
    +        }
    +        final Map topologyConf =
    +            
ConfigUtils.overrideLoginConfigWithSystemProperty(ConfigUtils.readSupervisorStormConf(conf,
 topologyId));
    +        List<ACL> acls = Utils.getWorkerACL(topologyConf);
    +        IStateStorage stateStorage =
    +            ClusterUtils.mkStateStorage(conf, topologyConf, acls, new 
ClusterStateContext(DaemonType.WORKER));
    +        IStormClusterState stormClusterState =
    +            ClusterUtils.mkStormClusterState(stateStorage, acls, new 
ClusterStateContext());
    +        Credentials initialCredentials = 
stormClusterState.credentials(topologyId, null);
    +        autoCreds = AuthUtils.GetAutoCredentials(topologyConf);
    +        subject = AuthUtils.populateSubject(null, autoCreds, 
initialCredentials.get_creds());
    +
    +        Subject.doAs(subject, new PrivilegedExceptionAction<Object>() {
    +            @Override public Object run() throws Exception {
    +                workerState =
    +                    new WorkerState(conf, context, topologyId, 
assignmentId, port, workerId, topologyConf, stateStorage,
    +                        stormClusterState);
    +
    +                // Heartbeat here so that worker process dies if this fails
    +                // it's important that worker heartbeat to supervisor ASAP 
so that supervisor knows
    +                // that worker is running and moves on
    +                doHeartBeat();
    +
    +                executorsAtom = new AtomicReference<>(null);
    +
    +                // launch heartbeat threads immediately so that 
slow-loading tasks don't cause the worker to timeout
    +                // to the supervisor
    +                workerState.heartbeatTimer
    +                    .scheduleRecurring(0, (Integer) 
conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS), () -> {
    +                        try {
    +                            doHeartBeat();
    +                        } catch (IOException e) {
    +                            throw new RuntimeException(e);
    +                        }
    +                    });
    +
    +                workerState.executorHeartbeatTimer
    +                    .scheduleRecurring(0, (Integer) 
conf.get(Config.WORKER_HEARTBEAT_FREQUENCY_SECS),
    +                        Worker.this::doExecutorHeartbeats);
    +
    +                workerState.registerCallbacks();
    +
    +                workerState.refreshConnections(null);
    +
    +                workerState.activateWorkerWhenAllConnectionsReady();
    +
    +                workerState.refreshStormActive(null);
    +
    +                workerState.runWorkerStartHooks();
    +
    +                List<IRunningExecutor> newExecutors = new 
ArrayList<IRunningExecutor>();
    +                for (List<Long> e : workerState.getExecutors()) {
    +                    if (ConfigUtils.isLocalMode(topologyConf)) {
    +                        newExecutors.add(
    +                            LocalExecutor.mkExecutor(workerState, e, 
initialCredentials.get_creds())
    +                                .execute());
    +                    } else {
    +                        newExecutors.add(
    +                            Executor.mkExecutor(workerState, e, 
initialCredentials.get_creds())
    +                                .execute());
    +                    }
    +                }
    +                executorsAtom.set(newExecutors);
    +
    +                EventHandler<Object> tupleHandler = (packets, seqId, 
batchEnd) -> workerState
    +                    .sendTuplesToRemoteWorker((HashMap<Integer, 
ArrayList<TaskMessage>>) packets, seqId, batchEnd);
    +
    +                // This thread will publish the messages destined for 
remote tasks to remote connections
    +                transferThread = Utils.asyncLoop(() -> {
    +                    
workerState.transferQueue.consumeBatchWhenAvailable(tupleHandler);
    +                    return 0L;
    +                });
    +
    +                DisruptorBackpressureCallback disruptorBackpressureHandler 
=
    +                    mkDisruptorBackpressureHandler(workerState);
    +                
workerState.transferQueue.registerBackpressureCallback(disruptorBackpressureHandler);
    +                workerState.transferQueue
    +                    .setEnableBackpressure((Boolean) 
topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE));
    +                workerState.transferQueue
    +                    
.setHighWaterMark(Utils.getDouble(topologyConf.get(Config.BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK)));
    +                workerState.transferQueue
    +                    
.setLowWaterMark(Utils.getDouble(topologyConf.get(Config.BACKPRESSURE_DISRUPTOR_LOW_WATERMARK)));
    +
    +                WorkerBackpressureCallback backpressureCallback = 
mkBackpressureHandler();
    +                backpressureThread = new 
WorkerBackpressureThread(workerState.backpressureTrigger, workerState, 
backpressureCallback);
    +                if ((Boolean) 
topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE)) {
    +                    backpressureThread.start();
    +                    stormClusterState.topologyBackpressure(topologyId, 
workerState::refreshThrottle);
    +                }
    +
    +                credentialsAtom = new 
AtomicReference<Credentials>(initialCredentials);
    +
    +                establishLogSettingCallback();
    +
    +                workerState.stormClusterState.credentials(topologyId, 
Worker.this::checkCredentialsChanged);
    +
    +                workerState.refreshCredentialsTimer.scheduleRecurring(0,
    +                    (Integer) conf.get(Config.TASK_CREDENTIALS_POLL_SECS), 
new Runnable() {
    +                        @Override public void run() {
    +                            checkCredentialsChanged();
    +                            if ((Boolean) 
topologyConf.get(Config.TOPOLOGY_BACKPRESSURE_ENABLE)) {
    --- End diff --
    
    Before porting, we have another timer `refresh-backpressure-timer` which 
runs every `TASK_BACKPRESSURE_POLL_SECS`. 
    
    ```
        (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)         
          (.scheduleRecurring (:refresh-backpressure-timer worker) 0 (conf 
TASK-BACKPRESSURE-POLL-SECS) topology-backpressure-callback))
    ```
    
    IMHO it would be better to keep 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.
---

Reply via email to