@SuppressWarnings({"rawtypes", "unchecked"}) public WorkerData( Map conf, IContext context, String topology_id, String supervisor_id, int port, String worker_id, String jar_path) throws Exception { this.conf = conf; this.context = context; this.topologyId = topology_id; this.supervisorId = supervisor_id; this.port = port; this.workerId = worker_id; this.active = new AtomicBoolean(true); this.topologyStatus = StatusType.active; if (StormConfig.cluster_mode(conf).equals("distributed")) { String pidDir = StormConfig.worker_pids_root(conf, worker_id); JStormServerUtils.createPid(pidDir); } // create zk interface this.zkClusterstate = ZkTool.mk_distributed_cluster_state(conf); this.zkCluster = Cluster.mk_storm_cluster_state(zkClusterstate); Map rawConf = StormConfig.read_supervisor_topology_conf(conf, topology_id); this.stormConf = new HashMap<Object, Object>(); this.stormConf.putAll(conf); this.stormConf.putAll(rawConf); LOG.info("Worker Configuration " + stormConf); try { boolean enableClassloader = ConfigExtension.isEnableTopologyClassLoader(stormConf); boolean enableDebugClassloader = ConfigExtension.isEnableClassloaderDebug(stormConf); if (jar_path == null && enableClassloader == true) { LOG.error("enable classloader, but not app jar"); throw new InvalidParameterException(); } URL[] urlArray = new URL[0]; if (jar_path != null) { String[] paths = jar_path.split(":"); Set<URL> urls = new HashSet<URL>(); for (String path : paths) { if (StringUtils.isBlank(path)) continue; URL url = new URL("File:" + path); urls.add(url); } urlArray = urls.toArray(new URL[0]); } WorkerClassLoader.mkInstance( urlArray, ClassLoader.getSystemClassLoader(), ClassLoader.getSystemClassLoader().getParent(), enableClassloader, enableDebugClassloader); } catch (Exception e) { // TODO Auto-generated catch block LOG.error("init jarClassLoader error!", e); throw new InvalidParameterException(); } if (this.context == null) { this.context = TransportFactory.makeContext(stormConf); } boolean disruptorUseSleep = ConfigExtension.isDisruptorUseSleep(stormConf); DisruptorQueue.setUseSleep(disruptorUseSleep); boolean isLimited = ConfigExtension.getTopologyBufferSizeLimited(stormConf); DisruptorQueue.setLimited(isLimited); LOG.info("Disruptor use sleep:" + disruptorUseSleep + ", limited size:" + isLimited); // this.transferQueue = new LinkedBlockingQueue<TransferData>(); int buffer_size = Utils.getInt(conf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE)); WaitStrategy waitStrategy = (WaitStrategy) Utils.newInstance((String) conf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); this.transferQueue = DisruptorQueue.mkInstance("TotalTransfer", ProducerType.MULTI, buffer_size, waitStrategy); this.transferQueue.consumerStarted(); this.sendingQueue = DisruptorQueue.mkInstance("TotalSending", ProducerType.MULTI, buffer_size, waitStrategy); this.sendingQueue.consumerStarted(); this.nodeportSocket = new ConcurrentHashMap<WorkerSlot, IConnection>(); this.taskNodeport = new ConcurrentHashMap<Integer, WorkerSlot>(); this.workerToResource = new ConcurrentSkipListSet<ResourceWorkerSlot>(); this.innerTaskTransfer = new ConcurrentHashMap<Integer, DisruptorQueue>(); this.deserializeQueues = new ConcurrentHashMap<Integer, DisruptorQueue>(); Assignment assignment = zkCluster.assignment_info(topologyId, null); if (assignment == null) { String errMsg = "Failed to get Assignment of " + topologyId; LOG.error(errMsg); throw new RuntimeException(errMsg); } workerToResource.addAll(assignment.getWorkers()); // get current worker's task list this.taskids = assignment.getCurrentWorkerTasks(supervisorId, port); if (taskids.size() == 0) { throw new RuntimeException("No tasks running current workers"); } LOG.info("Current worker taskList:" + taskids); // deserialize topology code from local dir rawTopology = StormConfig.read_supervisor_topology_code(conf, topology_id); sysTopology = Common.system_topology(stormConf, rawTopology); generateMaps(); contextMaker = new ContextMaker(this); metricReporter = new MetricReporter(this); outTaskStatus = new HashMap<Integer, Boolean>(); threadPool = Executors.newScheduledThreadPool(THREAD_POOL_NUM); TimerTrigger.setScheduledExecutorService(threadPool); LOG.info("Successfully create WorkerData"); }
/** * create and start one supervisor * * @param conf : configurationdefault.yaml storm.yaml * @param sharedContext : null (right now) * @return SupervisorManger: which is used to shutdown all workers and supervisor */ @SuppressWarnings("rawtypes") public SupervisorManger mkSupervisor(Map conf, IContext sharedContext) throws Exception { LOG.info("Starting Supervisor with conf " + conf); active = new AtomicBoolean(true); /** Step 1: cleanup all files in /storm-local-dir/supervisor/tmp */ String path = StormConfig.supervisorTmpDir(conf); FileUtils.cleanDirectory(new File(path)); /* * Step 2: create ZK operation instance StromClusterState */ StormClusterState stormClusterState = Cluster.mk_storm_cluster_state(conf); /* * Step 3, create LocalStat LocalStat is one KV database 4.1 create * LocalState instance 4.2 get supervisorId, if no supervisorId, create * one */ LocalState localState = StormConfig.supervisorState(conf); String supervisorId = (String) localState.get(Common.LS_ID); if (supervisorId == null) { supervisorId = UUID.randomUUID().toString(); localState.put(Common.LS_ID, supervisorId); } Vector<SmartThread> threads = new Vector<SmartThread>(); // Step 5 create HeartBeat // every supervisor.heartbeat.frequency.secs, write SupervisorInfo to ZK String myHostName = null; myHostName = ConfigExtension.getSupervisorHost(conf); if (myHostName == null) { myHostName = NetWorkUtils.hostname(); } Heartbeat hb = new Heartbeat(conf, stormClusterState, supervisorId, myHostName, active); hb.update(); AsyncLoopThread heartbeat = new AsyncLoopThread(hb, false, null, Thread.MIN_PRIORITY, true); threads.add(heartbeat); // Step 6 create and start sync Supervisor thread // every supervisor.monitor.frequency.secs second run SyncSupervisor EventManager processEventManager = new EventManagerImp(false); ConcurrentHashMap<String, String> workerThreadPids = new ConcurrentHashMap<String, String>(); SyncProcessEvent syncProcessEvent = new SyncProcessEvent(supervisorId, conf, localState, workerThreadPids, sharedContext); EventManager syncSupEventManager = new EventManagerImp(false); SyncSupervisorEvent syncSupervisorEvent = new SyncSupervisorEvent( supervisorId, conf, processEventManager, syncSupEventManager, stormClusterState, localState, syncProcessEvent); int syncFrequence = JStormUtils.parseInt(conf.get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)); EventManagerPusher syncSupervisorPusher = new EventManagerPusher(syncSupEventManager, syncSupervisorEvent, active, syncFrequence); AsyncLoopThread syncSupervisorThread = new AsyncLoopThread(syncSupervisorPusher); threads.add(syncSupervisorThread); // Step 7 start sync process thread // every supervisor.monitor.frequency.secs run SyncProcesses // skip thread to do syncProcess, due to nimbus will check whether // worker is dead or not, if dead, it will reassign a new worker // // int syncProcessFrequence = syncFrequence/2; // EventManagerPusher syncProcessPusher = new EventManagerPusher( // processEventManager, syncProcessEvent, active, // syncProcessFrequence); // AsyncLoopThread syncProcessThread = new // AsyncLoopThread(syncProcessPusher); // threads.add(syncProcessThread); // Step 7 start httpserver Httpserver httpserver = new Httpserver(conf); httpserver.start(); LOG.info("Starting supervisor with id " + supervisorId + " at host " + myHostName); // SupervisorManger which can shutdown all supervisor and workers return new SupervisorManger( conf, supervisorId, active, threads, syncSupEventManager, processEventManager, httpserver, stormClusterState, workerThreadPids); }