public class LocalCluster extends Object implements ILocalClusterTrackedTopologyAware, Nimbus.Iface
LocalCluster is an AutoCloseable so if you are using it in tests you can use a try block to be sure it is shut down.
try (LocalCluster cluster = new LocalCluster()) { // Do some tests } // The cluster has been shut down.| Modifier and Type | Class and Description |
|---|---|
static class |
LocalCluster.Builder
Simple way to configure a LocalCluster to meet your needs.
|
class |
LocalCluster.LocalTopology
When running a topology locally, for tests etc.
|
ILocalCluster.ILocalTopology| Modifier and Type | Field and Description |
|---|---|
static KillOptions |
KILL_NOW |
| Constructor and Description |
|---|
LocalCluster()
Create a default LocalCluster.
|
LocalCluster(String zkHost,
Long zkPort)
Create a LocalCluster that connects to an existing Zookeeper instance.
|
| Modifier and Type | Method and Description |
|---|---|
void |
activate(String topologyName)
Activate a topology.
|
Supervisor |
addSupervisor()
Add another supervisor to the topology.
|
Supervisor |
addSupervisor(Number ports)
Add another supervisor to the topology.
|
Supervisor |
addSupervisor(Number ports,
Map<String,Object> conf,
String id)
Add another supervisor to the topology.
|
Supervisor |
addSupervisor(Number ports,
String id)
Add another supervisor to the topology.
|
void |
advanceClusterTime(int secs)
Advance the cluster time when the cluster is using SimulatedTime.
|
void |
advanceClusterTime(int secs,
int incSecs)
Advance the cluster time when the cluster is using SimulatedTime.
|
BeginDownloadResult |
beginBlobDownload(String key) |
String |
beginCreateBlob(String key,
SettableBlobMeta meta) |
String |
beginFileUpload() |
String |
beginUpdateBlob(String key) |
void |
cancelBlobUpload(String session) |
void |
close() |
void |
createStateInZookeeper(String key) |
void |
deactivate(String topologyName)
Deactivate a topology.
|
void |
debug(String name,
String component,
boolean enable,
double samplingPercentage)
Enable/disable logging the tuples generated in topology via an internal EventLogger bolt.
|
void |
deleteBlob(String key) |
ByteBuffer |
downloadBlobChunk(String session) |
ByteBuffer |
downloadChunk(String id) |
void |
finishBlobUpload(String session) |
void |
finishFileUpload(String location) |
ReadableBlobMeta |
getBlobMeta(String key) |
int |
getBlobReplication(String key) |
ClusterSummary |
getClusterInfo()
Get cluster information.
|
IStormClusterState |
getClusterState()
This is intended for internal testing only.
|
ComponentPageInfo |
getComponentPageInfo(String topologyId,
String componentId,
String window,
boolean isIncludeSys) |
List<ProfileRequest> |
getComponentPendingProfileActions(String id,
String componentId,
ProfileAction action) |
Map<String,Object> |
getDaemonConf()
Get daemon configuration.
|
NimbusSummary |
getLeader() |
LogConfig |
getLogConfig(String name) |
StormMetricsRegistry |
getMetricRegistry()
Reference to metrics registry.
|
Nimbus |
getNimbus()
Reference to nimbus.
|
String |
getNimbusConf() |
List<OwnerResourceSummary> |
getOwnerResourceSummaries(String owner) |
Supervisor |
getSupervisor(String id)
Get a specific Supervisor.
|
SupervisorAssignments |
getSupervisorAssignments(String node)
Get assigned assignments for a specific supervisor
|
SupervisorPageInfo |
getSupervisorPageInfo(String id,
String host,
boolean isIncludeSys) |
int |
getThriftServerPort() |
StormTopology |
getTopology(String id)
Get the compiled storm topology.
|
String |
getTopologyConf(String id)
The config of a topology as a JSON string.
|
TopologyHistoryInfo |
getTopologyHistory(String user) |
TopologyInfo |
getTopologyInfo(String id)
Get the state of a topology.
|
TopologyInfo |
getTopologyInfoByName(String name)
Get the state of a topology.
|
TopologyInfo |
getTopologyInfoByNameWithOpts(String name,
GetInfoOptions options)
Get the state of a topology.
|
TopologyInfo |
getTopologyInfoWithOpts(String id,
GetInfoOptions options)
Get the state of a topology.
|
TopologyPageInfo |
getTopologyPageInfo(String id,
String window,
boolean isIncludeSys) |
List<TopologySummary> |
getTopologySummaries() |
TopologySummary |
getTopologySummary(String id) |
TopologySummary |
getTopologySummaryByName(String name) |
String |
getTrackedId()
If the cluster is tracked get the id for the tracked cluster.
|
StormTopology |
getUserTopology(String id)
Returns the user specified topology as submitted originally.
|
boolean |
isRemoteBlobExists(String blobKey)
Decide if the blob is removed from cluster.
|
boolean |
isTopologyNameAllowed(String name) |
void |
killSupervisor(String id)
Kill a specific supervisor.
|
void |
killTopology(String topologyName)
Kill a topology (if you are not using ILocalTopology).
|
void |
killTopologyWithOpts(String name,
KillOptions options)
Kill a topology (if you are not using ILocalTopology).
|
ListBlobsResult |
listBlobs(String session) |
static void |
main(String[] args)
Main entry point to running in local mode.
|
void |
processWorkerMetrics(WorkerMetrics metrics) |
void |
rebalance(String name,
RebalanceOptions options)
Rebalance a topology.
|
void |
sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat)
Send supervisor local worker heartbeat when a supervisor is unreachable
|
void |
sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats)
Send supervisor worker heartbeats for a specific supervisor
|
void |
setBlobMeta(String key,
SettableBlobMeta meta) |
void |
setLogConfig(String name,
LogConfig config) |
void |
setWorkerProfiler(String id,
ProfileRequest profileRequest) |
void |
shutdown()
Shut down the cluster.
|
LocalCluster.LocalTopology |
submitTopology(String topologyName,
Map<String,Object> conf,
StormTopology topology)
Submit a topology to be run in local mode.
|
LocalCluster.LocalTopology |
submitTopology(String topologyName,
Map<String,Object> conf,
TrackedTopology topology)
Submit a tracked topology to be run in local mode.
|
void |
submitTopology(String name,
String uploadedJarLocation,
String jsonConf,
StormTopology topology) |
LocalCluster.LocalTopology |
submitTopologyWithOpts(String topologyName,
Map<String,Object> conf,
StormTopology topology,
SubmitOptions submitOpts)
Submit a topology to be run in local mode.
|
LocalCluster.LocalTopology |
submitTopologyWithOpts(String topologyName,
Map<String,Object> conf,
TrackedTopology topology,
SubmitOptions submitOpts)
Submit a tracked topology to be run in local mode.
|
void |
submitTopologyWithOpts(String name,
String uploadedJarLocation,
String jsonConf,
StormTopology topology,
SubmitOptions options) |
int |
updateBlobReplication(String key,
int replication) |
void |
uploadBlobChunk(String session,
ByteBuffer chunk) |
void |
uploadChunk(String location,
ByteBuffer chunk) |
void |
uploadNewCredentials(String topologyName,
Credentials creds)
Upload new credentials to a topology.
|
void |
waitForIdle()
Wait for the cluster to be idle.
|
void |
waitForIdle(long timeoutMs)
Wait for the cluster to be idle.
|
static <T> T |
withLocalModeOverride(Callable<T> c,
long ttlSec)
Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls.
|
static <T> T |
withLocalModeOverride(Callable<T> c,
long ttlSec,
Map<String,Object> daemonConf)
Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls.
|
public static final KillOptions KILL_NOW
public LocalCluster()
throws Exception
Exception - on any errorpublic static <T> T withLocalModeOverride(Callable<T> c, long ttlSec) throws Exception
c - the callable to run in this modettlSec - the number of seconds to let the cluster run after c has completedException - on any Exception.public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec, Map<String,Object> daemonConf) throws Exception
c - the callable to run in this modettlSec - the number of seconds to let the cluster run after c has completeddaemonConf - configs to set for the daemon processes.Exception - on any Exception.public static void main(String[] args) throws Exception
args - arguments to be run in local modeException - on any error when running.public Nimbus getNimbus()
public StormMetricsRegistry getMetricRegistry()
public Map<String,Object> getDaemonConf()
public LocalCluster.LocalTopology submitTopology(String topologyName, Map<String,Object> conf, StormTopology topology) throws org.apache.storm.thrift.TException
ILocalClustersubmitTopology in interface ILocalClustertopologyName - the name of the topology to useconf - the config for the topologytopology - the topology itself.org.apache.storm.thrift.TException - on any error from nimbuspublic LocalCluster.LocalTopology submitTopology(String topologyName, Map<String,Object> conf, TrackedTopology topology) throws org.apache.storm.thrift.TException
ILocalClusterTrackedTopologyAwaresubmitTopology in interface ILocalClusterTrackedTopologyAwaretopologyName - the name of the topology to useconf - the config for the topologytopology - the topology itself.org.apache.storm.thrift.TException - on any error from nimbuspublic void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.storm.thrift.TException
submitTopology in interface Nimbus.IfaceAlreadyAliveExceptionInvalidTopologyExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic LocalCluster.LocalTopology submitTopologyWithOpts(String topologyName, Map<String,Object> conf, StormTopology topology, SubmitOptions submitOpts) throws org.apache.storm.thrift.TException
ILocalClustersubmitTopologyWithOpts in interface ILocalClustertopologyName - the name of the topology to useconf - the config for the topologytopology - the topology itself.submitOpts - options for topologyorg.apache.storm.thrift.TException - on any error from nimbuspublic LocalCluster.LocalTopology submitTopologyWithOpts(String topologyName, Map<String,Object> conf, TrackedTopology topology, SubmitOptions submitOpts) throws org.apache.storm.thrift.TException
ILocalClusterTrackedTopologyAwaresubmitTopologyWithOpts in interface ILocalClusterTrackedTopologyAwaretopologyName - the name of the topology to useconf - the config for the topologytopology - the topology itself.submitOpts - options for topologyorg.apache.storm.thrift.TException - on any error from nimbuspublic void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, org.apache.storm.thrift.TException
submitTopologyWithOpts in interface Nimbus.IfaceAlreadyAliveExceptionInvalidTopologyExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void uploadNewCredentials(String topologyName, Credentials creds) throws org.apache.storm.thrift.TException
ILocalClusteruploadNewCredentials in interface Nimbus.IfaceuploadNewCredentials in interface ILocalClustertopologyName - the name of the topologycreds - the new credentails for the topology.org.apache.storm.thrift.TException - on any error from nimbuspublic void killTopology(String topologyName) throws org.apache.storm.thrift.TException
ILocalClusterkillTopology in interface Nimbus.IfacekillTopology in interface ILocalClustertopologyName - the name of the topologyorg.apache.storm.thrift.TException - on any error from nimbuspublic void killTopologyWithOpts(String name, KillOptions options) throws org.apache.storm.thrift.TException
ILocalClusterkillTopologyWithOpts in interface Nimbus.IfacekillTopologyWithOpts in interface ILocalClusteroptions - for how to kill the topologyorg.apache.storm.thrift.TException - on any error from nimbuspublic void activate(String topologyName) throws org.apache.storm.thrift.TException
ILocalClusteractivate in interface Nimbus.Ifaceactivate in interface ILocalClustertopologyName - the name of the topology to activateorg.apache.storm.thrift.TException - on any error from nimbuspublic void deactivate(String topologyName) throws org.apache.storm.thrift.TException
ILocalClusterdeactivate in interface Nimbus.Ifacedeactivate in interface ILocalClustertopologyName - the name of the topology to deactivateorg.apache.storm.thrift.TException - on any error from nimbuspublic void rebalance(String name, RebalanceOptions options) throws org.apache.storm.thrift.TException
ILocalClusterrebalance in interface Nimbus.Ifacerebalance in interface ILocalClustername - the name of the topologyoptions - options for rebalanceing the topology.org.apache.storm.thrift.TException - on any error from nimbuspublic void shutdown()
ILocalClustershutdown in interface ILocalClusterpublic String getTopologyConf(String id) throws org.apache.storm.thrift.TException
ILocalClustergetTopologyConf in interface Nimbus.IfacegetTopologyConf in interface ILocalClusterid - the id of the topology (not the name)org.apache.storm.thrift.TException - on any error from nimbuspublic StormTopology getTopology(String id) throws org.apache.storm.thrift.TException
ILocalClustergetTopology in interface Nimbus.IfacegetTopology in interface ILocalClusterid - the id of the topology (not the name)org.apache.storm.thrift.TException - on any error from nimbuspublic ClusterSummary getClusterInfo() throws org.apache.storm.thrift.TException
ILocalClustergetClusterInfo in interface Nimbus.IfacegetClusterInfo in interface ILocalClusterorg.apache.storm.thrift.TException - on any error from nimbuspublic List<TopologySummary> getTopologySummaries() throws org.apache.storm.thrift.TException
getTopologySummaries in interface Nimbus.IfacegetTopologySummaries in interface ILocalClusterorg.apache.storm.thrift.TExceptionpublic TopologySummary getTopologySummaryByName(String name) throws org.apache.storm.thrift.TException
getTopologySummaryByName in interface Nimbus.IfacegetTopologySummaryByName in interface ILocalClusterorg.apache.storm.thrift.TExceptionpublic TopologySummary getTopologySummary(String id) throws org.apache.storm.thrift.TException
getTopologySummary in interface Nimbus.IfacegetTopologySummary in interface ILocalClusterorg.apache.storm.thrift.TExceptionpublic TopologyInfo getTopologyInfo(String id) throws org.apache.storm.thrift.TException
ILocalClustergetTopologyInfo in interface Nimbus.IfacegetTopologyInfo in interface ILocalClusterid - the id of the topology (not the name)org.apache.storm.thrift.TException - on any error from nimbuspublic TopologyInfo getTopologyInfoByName(String name) throws org.apache.storm.thrift.TException
ILocalClustergetTopologyInfoByName in interface Nimbus.IfacegetTopologyInfoByName in interface ILocalClustername - the name of the topology (not the id)org.apache.storm.thrift.TException - on any error from nimbuspublic TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws org.apache.storm.thrift.TException
ILocalClustergetTopologyInfoWithOpts in interface Nimbus.IfacegetTopologyInfoWithOpts in interface ILocalClusterid - the id of the topology (not the name)options - This is to choose number of Error(s) in TopologyInfo.org.apache.storm.thrift.TException - on any error from nimbuspublic TopologyInfo getTopologyInfoByNameWithOpts(String name, GetInfoOptions options) throws org.apache.storm.thrift.TException
ILocalClustergetTopologyInfoByNameWithOpts in interface Nimbus.IfacegetTopologyInfoByNameWithOpts in interface ILocalClustername - the name of the topology (not the id)options - This is GetInfoOptions to choose Error(s) in on TopologyInfo.org.apache.storm.thrift.TException - on any error from nimbuspublic int getThriftServerPort()
public boolean isRemoteBlobExists(String blobKey) throws AuthorizationException, org.apache.storm.thrift.TException
Nimbus.IfaceisRemoteBlobExists in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void close()
throws Exception
close in interface AutoCloseableExceptionpublic Supervisor getSupervisor(String id)
id - the id of the supervisorpublic void killSupervisor(String id)
id - the id of the supervisorpublic Supervisor addSupervisor() throws Exception
Exceptionpublic Supervisor addSupervisor(Number ports) throws Exception
ports - the number of ports/slots the supervisor should haveExceptionpublic Supervisor addSupervisor(Number ports, String id) throws Exception
ports - the number of ports/slots the supervisor should haveid - the id of the new supervisor, so you can find it later.Exceptionpublic Supervisor addSupervisor(Number ports, Map<String,Object> conf, String id) throws Exception
ports - the number of ports/slots the supervisor should haveconf - any config values that should be added/over written in the daemon conf of the cluster.id - the id of the new supervisor, so you can find it later.Exceptionpublic void waitForIdle()
throws InterruptedException
InterruptedException - if interrupted while waiting.AssertionError - if the cluster did not come to an idle point with a timeout.public void waitForIdle(long timeoutMs)
throws InterruptedException
timeoutMs - the number of ms to wait before throwing an error.InterruptedException - if interrupted while waiting.AssertionError - if the cluster did not come to an idle point with a timeout.public void advanceClusterTime(int secs)
throws InterruptedException
ILocalClusteradvanceClusterTime in interface ILocalClustersecs - the number of seconds to advance timeInterruptedExceptionpublic void advanceClusterTime(int secs,
int incSecs)
throws InterruptedException
ILocalClusteradvanceClusterTime in interface ILocalClustersecs - the number of seconds to advance timeInterruptedExceptionpublic IStormClusterState getClusterState()
ILocalClustergetClusterState in interface ILocalClusterpublic String getTrackedId()
ILocalClustergetTrackedId in interface ILocalClusterpublic void setLogConfig(String name, LogConfig config) throws org.apache.storm.thrift.TException
setLogConfig in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic LogConfig getLogConfig(String name) throws org.apache.storm.thrift.TException
getLogConfig in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.storm.thrift.TException
Nimbus.Ifacedebug in interface Nimbus.IfaceNotAliveExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.storm.thrift.TException
setWorkerProfiler in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic List<ProfileRequest> getComponentPendingProfileActions(String id, String componentId, ProfileAction action) throws org.apache.storm.thrift.TException
getComponentPendingProfileActions in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic String beginCreateBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException, org.apache.storm.thrift.TException
beginCreateBlob in interface Nimbus.IfaceAuthorizationExceptionKeyAlreadyExistsExceptionorg.apache.storm.thrift.TExceptionpublic String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
beginUpdateBlob in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, org.apache.storm.thrift.TException
uploadBlobChunk in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void finishBlobUpload(String session) throws AuthorizationException, org.apache.storm.thrift.TException
finishBlobUpload in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void cancelBlobUpload(String session) throws AuthorizationException, org.apache.storm.thrift.TException
cancelBlobUpload in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
getBlobMeta in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
setBlobMeta in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic BeginDownloadResult beginBlobDownload(String key) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
beginBlobDownload in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, org.apache.storm.thrift.TException
downloadBlobChunk in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
deleteBlob in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic ListBlobsResult listBlobs(String session) throws org.apache.storm.thrift.TException
listBlobs in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
getBlobReplication in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException, org.apache.storm.thrift.TException
updateBlobReplication in interface Nimbus.IfaceAuthorizationExceptionKeyNotFoundExceptionorg.apache.storm.thrift.TExceptionpublic void createStateInZookeeper(String key) throws org.apache.storm.thrift.TException
createStateInZookeeper in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionpublic String beginFileUpload() throws AuthorizationException, org.apache.storm.thrift.TException
beginFileUpload in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.storm.thrift.TException
uploadChunk in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void finishFileUpload(String location) throws AuthorizationException, org.apache.storm.thrift.TException
finishFileUpload in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic ByteBuffer downloadChunk(String id) throws AuthorizationException, org.apache.storm.thrift.TException
downloadChunk in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic String getNimbusConf() throws AuthorizationException, org.apache.storm.thrift.TException
getNimbusConf in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic NimbusSummary getLeader() throws AuthorizationException, org.apache.storm.thrift.TException
getLeader in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic boolean isTopologyNameAllowed(String name) throws AuthorizationException, org.apache.storm.thrift.TException
isTopologyNameAllowed in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic TopologyPageInfo getTopologyPageInfo(String id, String window, boolean isIncludeSys) throws NotAliveException, AuthorizationException, org.apache.storm.thrift.TException
getTopologyPageInfo in interface Nimbus.IfaceNotAliveExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic SupervisorPageInfo getSupervisorPageInfo(String id, String host, boolean isIncludeSys) throws NotAliveException, AuthorizationException, org.apache.storm.thrift.TException
getSupervisorPageInfo in interface Nimbus.IfaceNotAliveExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic ComponentPageInfo getComponentPageInfo(String topologyId, String componentId, String window, boolean isIncludeSys) throws NotAliveException, AuthorizationException, org.apache.storm.thrift.TException
getComponentPageInfo in interface Nimbus.IfaceNotAliveExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.storm.thrift.TException
Nimbus.IfaceNimbus.Iface.getTopology(String id).getUserTopology in interface Nimbus.IfaceNotAliveExceptionAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, org.apache.storm.thrift.TException
getTopologyHistory in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic List<OwnerResourceSummary> getOwnerResourceSummaries(String owner) throws AuthorizationException, org.apache.storm.thrift.TException
getOwnerResourceSummaries in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic SupervisorAssignments getSupervisorAssignments(String node) throws AuthorizationException, org.apache.storm.thrift.TException
Nimbus.IfacegetSupervisorAssignments in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void sendSupervisorWorkerHeartbeats(SupervisorWorkerHeartbeats heartbeats) throws AuthorizationException, org.apache.storm.thrift.TException
Nimbus.IfacesendSupervisorWorkerHeartbeats in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void sendSupervisorWorkerHeartbeat(SupervisorWorkerHeartbeat heatbeat) throws AuthorizationException, org.apache.storm.thrift.TException
Nimbus.IfacesendSupervisorWorkerHeartbeat in interface Nimbus.IfaceAuthorizationExceptionorg.apache.storm.thrift.TExceptionpublic void processWorkerMetrics(WorkerMetrics metrics) throws org.apache.storm.thrift.TException
processWorkerMetrics in interface Nimbus.Ifaceorg.apache.storm.thrift.TExceptionCopyright © 2023 The Apache Software Foundation. All rights reserved.