@Override public void disconnect() { stateStorage.unregister(stateId); if (solo) { stateStorage.close(); this.assignmentsBackend.close(); } }
@Override public void setTopologyLogConfig(String stormId, LogConfig logConfig, Map<String, Object> topoConf) { stateStorage.mkdirs(ClusterUtils.LOGCONFIG_SUBTREE, defaultAcls); stateStorage.set_data(ClusterUtils.logConfigPath(stormId), Utils.serialize(logConfig), ClusterUtils.mkTopoReadOnlyAcls(topoConf)); }
@Override public String register(ZKStateChangedCallback callback) { return stateStorage.register(callback); }
@Override public List<String> blobstore(Runnable callback) { if (callback != null) { blobstoreCallback.set(callback); } stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE); return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, 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); } }
errorInfo.set_port(port.intValue()); byte[] serData = Utils.serialize(errorInfo); stateStorage.mkdirs(path, defaultAcls); stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, defaultAcls); stateStorage.set_data(lastErrorPath, serData, defaultAcls); List<String> childrens = stateStorage.get_children(path, false); String znodePath = path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0); try { stateStorage.delete_node(znodePath); } catch (Exception e) { if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
@Override public List<ProfileRequest> getTopologyProfileRequests(String stormId) { List<ProfileRequest> profileRequests = new ArrayList<>(); String path = ClusterUtils.profilerConfigPath(stormId); if (stateStorage.node_exists(path, false)) { List<String> strs = stateStorage.get_children(path, false); for (String str : strs) { String childPath = path + ClusterUtils.ZK_SEPERATOR + str; byte[] raw = stateStorage.get_data(childPath, false); ProfileRequest request = ClusterUtils.maybeDeserialize(raw, ProfileRequest.class); if (request != null) { profileRequests.add(request); } } } return profileRequests; }
@Override public void setupBlob(String key, NimbusInfo nimbusInfo, Integer versionInfo) { String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo; LOG.info("set-path: {}", path); stateStorage.mkdirs(ClusterUtils.blobstorePath(key), defaultAcls); stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString()); stateStorage.set_ephemeral_node(path, null, defaultAcls); }
@Override public void mkdirs(String path, List<ACL> acls) { stateStorage.mkdirs(path, acls); }
@Override public void addNimbusHost(final String nimbusId, final NimbusSummary nimbusSummary) { // explicit delete for ephmeral node to ensure this session creates the entry. stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId)); stateStorage.add_listener((curatorFramework, connectionState) -> { LOG.info("Connection state listener invoked, zookeeper connection state has changed to {}", connectionState); if (connectionState.equals(ConnectionState.RECONNECTED)) { LOG.info("Connection state has changed to reconnected so setting nimbuses entry one more time"); // explicit delete for ephemeral node to ensure this session creates the entry. stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId)); stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), defaultAcls); } }); stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), defaultAcls); }
@Override public void delete_node(String path) { stateStorage.delete_node(path); }
@Override public List<String> get_children(String path, boolean watch) { return stateStorage.get_children(path, watch); }
@Override public byte[] get_data(String path, boolean watch) { return stateStorage.get_data(path, watch); }
@Override public List<NimbusSummary> nimbuses() { List<NimbusSummary> nimbusSummaries = new ArrayList<>(); List<String> nimbusIds = stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false); for (String nimbusId : nimbusIds) { byte[] serialized = stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false); // check for null which can exist because of a race condition in which nimbus nodes in zk may have been // removed when connections are reconnected after getting children in the above line if (serialized != null) { NimbusSummary nimbusSummary = ClusterUtils.maybeDeserialize(serialized, NimbusSummary.class); nimbusSummaries.add(nimbusSummary); } } return nimbusSummaries; }
@Override public void removeExpiredPrivateWorkerKeys(String topologyId) { for (WorkerTokenServiceType type : WorkerTokenServiceType.values()) { String basePath = ClusterUtils.secretKeysPath(type, topologyId); try { for (String version : stateStorage.get_children(basePath, false)) { String fullPath = basePath + ClusterUtils.ZK_SEPERATOR + version; try { PrivateWorkerKey key = ClusterUtils.maybeDeserialize(stateStorage.get_data(fullPath, false), PrivateWorkerKey.class); if (Time.currentTimeMillis() > key.get_expirationTimeMillis()) { LOG.info("Removing expired worker key {}", fullPath); stateStorage.delete_node(fullPath); } } catch (RuntimeException e) { //This should never happen because only the primary nimbus is active, but just in case // declare the race safe, even if we lose it. if (!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { throw e; } } } } catch (RuntimeException e) { //No node for basePath is OK, nothing to remove if (!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { throw e; } } } }
@Override public void close() { stateStorage.close(); pacemakerClientPool.close(); }
/** * if znode exists and to be not on?, delete; if exists and on?, do nothing; if not exists and to be on?, create; if not exists and not on?, do nothing; * * @param stormId * @param node * @param port * @param on */ @Override public void workerBackpressure(String stormId, String node, Long port, boolean on) { String path = ClusterUtils.backpressurePath(stormId, node, port); boolean existed = stateStorage.node_exists(path, false); if (existed) { if (on == false) stateStorage.delete_node(path); } else { if (on == true) { stateStorage.set_ephemeral_node(path, null, acls); } } }
@Override public ErrorInfo lastError(String stormId, String componentId) { String path = ClusterUtils.lastErrorPath(stormId, componentId); if (stateStorage.node_exists(path, false)) { ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class); return errorInfo; } return null; }
@Override public PrivateWorkerKey getPrivateWorkerKey(WorkerTokenServiceType type, String topologyId, long keyVersion) { String path = ClusterUtils.secretKeysPath(type, topologyId, keyVersion); byte[] data = stateStorage.get_data(path, false); if (data == null) { LOG.debug("Could not find entry at {} will sync to see if that fixes it", path); //We didn't find it, but there are races, so we want to check again after a sync stateStorage.sync_path(path); data = stateStorage.get_data(path, false); } return ClusterUtils.maybeDeserialize(data, PrivateWorkerKey.class); }