@Override public <T> Object execute(T... args) { boolean isSetTaskInfo = false; try { Boolean reassign = (Boolean) args[1]; Map<Object, Object> conf = (Map<Object, Object>) args[2]; // args[0]: // delay, // args[1]: // reassign_flag, // args[2]: // conf if (conf != null) { boolean isConfUpdate = false; Map stormConf = data.getConf(); // Update topology code Map topoConf = StormConfig.read_nimbus_topology_conf(stormConf, topologyid); StormTopology rawOldTopology = StormConfig.read_nimbus_topology_code(stormConf, topologyid); StormTopology rawNewTopology = NimbusUtils.normalizeTopology(conf, rawOldTopology, true); StormTopology sysOldTopology = rawOldTopology.deepCopy(); StormTopology sysNewTopology = rawNewTopology.deepCopy(); if (conf.get(Config.TOPOLOGY_ACKER_EXECUTORS) != null) { Common.add_acker(topoConf, sysOldTopology); Common.add_acker(conf, sysNewTopology); int ackerNum = JStormUtils.parseInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS)); int oldAckerNum = JStormUtils.parseInt(topoConf.get(Config.TOPOLOGY_ACKER_EXECUTORS)); LOG.info("Update acker from oldAckerNum=" + oldAckerNum + " to ackerNum=" + ackerNum); topoConf.put(Config.TOPOLOGY_ACKER_EXECUTORS, ackerNum); isConfUpdate = true; } // If scale-out, setup task info for new added tasks setTaskInfo(sysOldTopology, sysNewTopology); isSetTaskInfo = true; // If everything is OK, write topology code into disk StormConfig.write_nimbus_topology_code( stormConf, topologyid, Utils.serialize(rawNewTopology)); // Update topology conf if worker num has been updated Set<Object> keys = conf.keySet(); Integer workerNum = JStormUtils.parseInt(conf.get(Config.TOPOLOGY_WORKERS)); if (workerNum != null) { Integer oldWorkerNum = JStormUtils.parseInt(topoConf.get(Config.TOPOLOGY_WORKERS)); topoConf.put(Config.TOPOLOGY_WORKERS, workerNum); isConfUpdate = true; LOG.info("Update worker num from " + oldWorkerNum + " to " + workerNum); } if (keys.contains(Config.ISOLATION_SCHEDULER_MACHINES)) { topoConf.put( Config.ISOLATION_SCHEDULER_MACHINES, conf.get(Config.ISOLATION_SCHEDULER_MACHINES)); } if (isConfUpdate) { StormConfig.write_nimbus_topology_conf(stormConf, topologyid, topoConf); } } TopologyAssignEvent event = new TopologyAssignEvent(); event.setTopologyId(topologyid); event.setScratch(true); event.setOldStatus(oldStatus); event.setReassign(reassign); if (conf != null) event.setScaleTopology(true); TopologyAssign.push(event); event.waitFinish(); } catch (Exception e) { LOG.error("do-rebalance error!", e); // Rollback the changes on ZK if (isSetTaskInfo) { try { StormClusterState clusterState = data.getStormClusterState(); clusterState.remove_task(topologyid, newTasks); } catch (Exception e1) { LOG.error("Failed to rollback the changes on ZK for task-" + newTasks, e); } } } DelayStatusTransitionCallback delayCallback = new DelayStatusTransitionCallback( data, topologyid, oldStatus, StatusType.rebalancing, StatusType.done_rebalance); return delayCallback.execute(); }
/** * Submit one Topology * * @param topologyname String: topology name * @param uploadedJarLocation String: already uploaded jar path * @param jsonConf String: jsonConf serialize all toplogy configuration to Json * @param topology StormTopology: topology Object */ @SuppressWarnings("unchecked") @Override public void submitTopologyWithOpts( String topologyname, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, TException { LOG.info("Receive " + topologyname + ", uploadedJarLocation:" + uploadedJarLocation); // @@@ Move validate topologyname in client code try { checkTopologyActive(data, topologyname, false); } catch (AlreadyAliveException e) { LOG.info(topologyname + " is already exist "); throw e; } catch (Exception e) { LOG.info("Failed to check whether topology is alive or not", e); throw new TException(e); } int counter = data.getSubmittedCount().incrementAndGet(); String topologyId = topologyname + "-" + counter + "-" + TimeUtils.current_time_secs(); Map<Object, Object> serializedConf = (Map<Object, Object>) JStormUtils.from_json(jsonConf); if (serializedConf == null) { LOG.warn("Failed to serialized Configuration"); throw new InvalidTopologyException("Failed to serilaze topology configuration"); } serializedConf.put(Config.TOPOLOGY_ID, topologyId); serializedConf.put(Config.TOPOLOGY_NAME, topologyname); try { Map<Object, Object> stormConf; stormConf = NimbusUtils.normalizeConf(conf, serializedConf, topology); Map<Object, Object> totalStormConf = new HashMap<Object, Object>(conf); totalStormConf.putAll(stormConf); StormTopology normalizedTopology = NimbusUtils.normalizeTopology(stormConf, topology); // this validates the structure of the topology Common.validate_basic(normalizedTopology, totalStormConf, topologyId); // don't need generate real topology, so skip Common.system_topology // Common.system_topology(totalStormConf, topology); StormClusterState stormClusterState = data.getStormClusterState(); // create /local-dir/nimbus/topologyId/xxxx files setupStormCode(conf, topologyId, uploadedJarLocation, stormConf, normalizedTopology); // generate TaskInfo for every bolt or spout in ZK // /ZK/tasks/topoologyId/xxx setupZkTaskInfo(conf, topologyId, stormClusterState); // make assignments for a topology TopologyAssignEvent assignEvent = new TopologyAssignEvent(); assignEvent.setTopologyId(topologyId); assignEvent.setScratch(false); assignEvent.setTopologyName(topologyname); assignEvent.setOldStatus( Thrift.topologyInitialStatusToStormStatus(options.get_initial_status())); TopologyAssign.push(assignEvent); LOG.info("Submit for " + topologyname + " with conf " + serializedConf); boolean isSuccess = assignEvent.waitFinish(); if (isSuccess == true) { LOG.info("Finish submit for " + topologyname); } else { throw new FailedAssignTopologyException(assignEvent.getErrorMsg()); } } catch (FailedAssignTopologyException e) { StringBuilder sb = new StringBuilder(); sb.append("Fail to sumbit topology, Root cause:"); if (e.getMessage() == null) { sb.append("submit timeout"); } else { sb.append(e.getMessage()); } sb.append("\n\n"); sb.append("topologyId:" + topologyId); sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); LOG.error(sb.toString(), e); throw new TopologyAssignException(sb.toString()); } catch (InvalidParameterException e) { StringBuilder sb = new StringBuilder(); sb.append("Fail to sumbit topology "); sb.append(e.getMessage()); sb.append(", cause:" + e.getCause()); sb.append("\n\n"); sb.append("topologyId:" + topologyId); sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); LOG.error(sb.toString(), e); throw new InvalidParameterException(sb.toString()); } catch (Throwable e) { StringBuilder sb = new StringBuilder(); sb.append("Fail to sumbit topology "); sb.append(e.getMessage()); sb.append(", cause:" + e.getCause()); sb.append("\n\n"); sb.append("topologyId:" + topologyId); sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); LOG.error(sb.toString(), e); throw new TopologyAssignException(sb.toString()); } }