private static IStormClusterState makeStormClusterState(Map<String, Object> conf) throws Exception { return ClusterUtils.mkStormClusterState(conf, new ClusterStateContext(DaemonType.NIMBUS, conf)); }
@Override public void setupErrors(String stormId, Map<String, Object> topoConf) { stateStorage.mkdirs(ClusterUtils.ERRORS_SUBTREE, defaultAcls); stateStorage.mkdirs(ClusterUtils.errorStormRoot(stormId), ClusterUtils.mkTopoReadWriteAcls(topoConf)); }
@Override public void removeStorm(String stormId) { stateStorage.delete_node(ClusterUtils.assignmentPath(stormId)); this.assignmentsBackend.clearStateForStorm(stormId); stateStorage.delete_node(ClusterUtils.credentialsPath(stormId)); stateStorage.delete_node(ClusterUtils.logConfigPath(stormId)); stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId)); removeStormBase(stormId); }
@Override public Assignment remoteAssignmentInfo(String stormId, Runnable callback) { if (callback != null) { assignmentInfoCallback.put(stormId, callback); } byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null); return ClusterUtils.maybeDeserialize(serialized, Assignment.class); }
@Override public Credentials credentials(String stormId, Runnable callback) { if (callback != null) { credentialsCallback.put(stormId, callback); } String path = ClusterUtils.credentialsPath(stormId); return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class); }
@Override public void setCredentials(String stormId, Credentials creds, Map<String, Object> topoConf) { List<ACL> aclList = ClusterUtils.mkTopoReadOnlyAcls(topoConf); String path = ClusterUtils.credentialsPath(stormId); stateStorage.set_data(path, Utils.serialize(creds), aclList); }
@Override public void reportError(String stormId, String componentId, String node, Long port, Throwable error) { String path = ClusterUtils.errorPath(stormId, componentId); String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId); ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.stringifyError(error), Time.currentTimeSecs()); errorInfo.set_host(node); errorInfo.set_port(port.intValue());
ConfigUtils.overrideLoginConfigWithSystemProperty(ConfigUtils.readSupervisorStormConf(conf, topologyId)); ClusterStateContext csContext = new ClusterStateContext(DaemonType.WORKER, topologyConf); IStateStorage stateStorage = ClusterUtils.mkStateStorage(conf, topologyConf, csContext); IStormClusterState stormClusterState = ClusterUtils.mkStormClusterState(stateStorage, null, csContext);
public static String errorPath(String stormId, String componentId) { try { return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8"); } catch (UnsupportedEncodingException e) { throw Utils.wrapInRuntime(e); } }
@Override public void setupBackpressure(String stormId, Map<String, Object> topoConf) { stateStorage.mkdirs(ClusterUtils.BACKPRESSURE_SUBTREE, defaultAcls); stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), ClusterUtils.mkTopoReadWriteAcls(topoConf)); }
@Override public void setAssignment(String stormId, Assignment info, Map<String, Object> topoConf) { byte[] serAssignment = Utils.serialize(info); stateStorage.mkdirs(ClusterUtils.ASSIGNMENTS_SUBTREE, defaultAcls); stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), ClusterUtils.mkTopoReadOnlyAcls(topoConf)); this.assignmentsBackend.keepOrUpdateAssignment(stormId, info); }
@Override public List<ErrorInfo> errors(String stormId, String componentId) { List<ErrorInfo> errorInfos = new ArrayList<>(); String path = ClusterUtils.errorPath(stormId, componentId); if (stateStorage.node_exists(path, false)) { List<String> childrens = stateStorage.get_children(path, false); for (String child : childrens) { String childPath = path + ClusterUtils.ZK_SEPERATOR + child; ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class); if (errorInfo != null) { errorInfos.add(errorInfo); } } } Collections.sort(errorInfos, new Comparator<ErrorInfo>() { public int compare(ErrorInfo arg0, ErrorInfo arg1) { return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs()); } }); return errorInfos; }
String path = ClusterUtils.errorStormRoot(topoId); if (zk.checkExists().forPath(path) == null) { LOG.warn("Creating missing errors location {}", path); verifyAclStrictRecursive(zk, superAcl, ClusterUtils.secretKeysPath(WorkerTokenServiceType.NIMBUS), fixUp); verifyAclStrictRecursive(zk, drpcFullAcl, ClusterUtils.secretKeysPath(WorkerTokenServiceType.DRPC), fixUp);
/** * Check whether a topology is in throttle-on status or not: if the backpresure/storm-id dir is not empty, this topology has * throttle-on, otherwise throttle-off. But if the backpresure/storm-id dir is not empty and has not been updated for more than * timeoutMs, we treat it as throttle-off. This will prevent the spouts from getting stuck indefinitely if something wrong happens. * * @param stormId The topology Id * @param timeoutMs How long until the backpressure znode is invalid. * @param callback The callback function * @return True is backpresure/storm-id dir is not empty and at least one of the backpressure znodes has not timed out; false otherwise. */ @Override public boolean topologyBackpressure(String stormId, long timeoutMs, Runnable callback) { if (callback != null) { backPressureCallback.put(stormId, callback); } String path = ClusterUtils.backpressureStormRoot(stormId); long mostRecentTimestamp = 0; if (stateStorage.node_exists(path, false)) { List<String> children = stateStorage.get_children(path, callback != null); mostRecentTimestamp = children.stream() .map(childPath -> stateStorage.get_data(ClusterUtils.backpressurePath(stormId, childPath), false)) .filter(data -> data != null) .mapToLong(data -> ByteBuffer.wrap(data).getLong()) .max() .orElse(0); } boolean ret = ((System.currentTimeMillis() - mostRecentTimestamp) < timeoutMs); LOG.debug("topology backpressure is {}", ret ? "on" : "off"); return ret; }
@Override public void removeBlobstoreKey(String blobKey) { LOG.debug("remove key {}", blobKey); stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey)); }
/** * Get the backpressure znode full path. * * @param stormId The topology id * @param shortPath A string in the form of "node-port" * @return The backpressure znode path */ public static String backpressurePath(String stormId, String shortPath) { return backpressureStormRoot(stormId) + ZK_SEPERATOR + shortPath; }
@Override public Integer assignmentVersion(String stormId, Runnable callback) throws Exception { if (callback != null) { assignmentVersionCallback.put(stormId, callback); } return stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != null); }
@Override public void removeWorkerBackpressure(String stormId, String node, Long port) { String path = ClusterUtils.backpressurePath(stormId, node, port); boolean existed = stateStorage.node_exists(path, false); if (existed) { stateStorage.delete_node(path); } }
@Override public void removeKeyVersion(String blobKey) { stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey)); }
public static String lastErrorPath(String stormId, String componentId) { return errorPath(stormId, componentId) + "-last-error"; }