Skip to content

Commit

Permalink
Merge pull request #3274 from bipinprasad/storm3639
Browse files Browse the repository at this point in the history
[STORM-3639] Replace asserts in daemon code.
jnioche authored Dec 4, 2023
2 parents c158dee + 5e9a2da commit 2785507
Showing 13 changed files with 320 additions and 137 deletions.
Original file line number Diff line number Diff line change
@@ -828,9 +828,15 @@ private static Map<String, Map<WorkerSlot, WorkerResources>> computeTopoToNodePo
return ret;
}

/**
* Check new assignments with existing assignments and determine difference is any.
*
* @param existingAssignments non-null map of topology-id to existing assignments.
* @param newAssignments non-null map of topology-id to new assignments.
* @return true if there is a change in assignments, false otherwise.
*/
private boolean auditAssignmentChanges(Map<String, Assignment> existingAssignments,
Map<String, Assignment> newAssignments) {
assert existingAssignments != null && newAssignments != null;
boolean anyChanged = existingAssignments.isEmpty() ^ newAssignments.isEmpty();
long numRemovedExec = 0;
long numRemovedSlot = 0;
@@ -1966,8 +1972,12 @@ private void waitForDesiredCodeReplication(Map<String, Object> topoConf, String

private TopologyDetails readTopologyDetails(String topoId, StormBase base) throws KeyNotFoundException,
AuthorizationException, IOException, InvalidTopologyException {
assert (base != null);
assert (topoId != null);
if (base == null) {
throw new InvalidTopologyException("Cannot readTopologyDetails: StormBase parameter value is null");
}
if (topoId == null) {
throw new InvalidTopologyException("Cannot readTopologyDetails: topoId parameter value is null");
}

Map<String, Object> topoConf = readTopoConfAsNimbus(topoId, topoCache);
StormTopology topo = readStormTopologyAsNimbus(topoId, topoCache);
@@ -2067,7 +2077,9 @@ private List<List<Integer>> computeExecutors(StormBase base, Map<String, Object>
StormTopology topology)
throws InvalidTopologyException {

assert (base != null);
if (base == null) {
throw new InvalidTopologyException("Cannot computeExecutors: StormBase parameter value is null");
}

Map<String, Integer> compToExecutors = base.get_component_executors();
List<List<Integer>> ret = new ArrayList<>();
@@ -2677,7 +2689,9 @@ private int getTopologyLaunchHeartbeatTimeoutSec(String topoId) {
private void startTopology(String topoName, String topoId, TopologyStatus initStatus, String owner,
String principal, Map<String, Object> topoConf, StormTopology stormTopology)
throws InvalidTopologyException {
assert (TopologyStatus.ACTIVE == initStatus || TopologyStatus.INACTIVE == initStatus);
if (TopologyStatus.ACTIVE != initStatus && TopologyStatus.INACTIVE != initStatus) {
throw new InvalidTopologyException("Cannot startTopology: initStatus should be ACTIVE or INACTIVE, not " + initStatus.name());
}
Map<String, Integer> numExecutors = new HashMap<>();
StormTopology topology = StormCommon.systemTopology(topoConf, stormTopology);
for (Entry<String, Object> entry : StormCommon.allComponents(topology).entrySet()) {
@@ -3195,7 +3209,9 @@ public void submitTopologyWithOpts(String topoName, String uploadedJarLocation,
try {
submitTopologyWithOptsCalls.mark();
assertIsLeader();
assert (options != null);
if (options == null) {
throw new InvalidTopologyException("Cannot submitTopologyWithOpts: SubmitOptions parameter value is null");
}
validateTopologyName(topoName);
checkAuthorization(topoName, null, "submitTopology");
assertTopoActive(topoName, false);
@@ -5321,10 +5337,12 @@ private class ClusterSummaryMetricSet implements Runnable {
ClusterSummaryMetricSet(StormMetricsRegistry metricsRegistry) {
this.metricsRegistry = metricsRegistry;
//Break the code if out of sync to thrift protocol
assert ClusterSummary._Fields.values().length == 3
&& ClusterSummary._Fields.findByName("supervisors") == ClusterSummary._Fields.SUPERVISORS
&& ClusterSummary._Fields.findByName("topologies") == ClusterSummary._Fields.TOPOLOGIES
&& ClusterSummary._Fields.findByName("nimbuses") == ClusterSummary._Fields.NIMBUSES;
if (ClusterSummary._Fields.values().length != 3
|| ClusterSummary._Fields.findByName("supervisors") != ClusterSummary._Fields.SUPERVISORS
|| ClusterSummary._Fields.findByName("topologies") != ClusterSummary._Fields.TOPOLOGIES
|| ClusterSummary._Fields.findByName("nimbuses") != ClusterSummary._Fields.NIMBUSES) {
throw new AssertionError("Out of sync with thrift protocol");
}

final CachedGauge<ClusterSummary> cachedSummary = new CachedGauge<ClusterSummary>(CACHING_WINDOW, TimeUnit.SECONDS) {
@Override
Original file line number Diff line number Diff line change
@@ -130,7 +130,9 @@ public BasicContainer(ContainerType type, Map<String, Object> conf, String super
AdvancedFSOps ops, String profileCmd) throws IOException {
super(type, conf, supervisorId, supervisorPort, port, assignment,
resourceIsolationManager, workerId, topoConf, ops, metricsRegistry, containerMemoryTracker);
assert (localState != null);
if (localState == null) {
throw new IOException("LocalState parameter value is null");
}
this.localState = localState;

if (type.isRecovery() && !type.isOnlyKillable()) {
@@ -209,7 +211,11 @@ public static String getStormVersionFor(final Map<String, Object> conf, final St
*/
protected void createNewWorkerId() {
type.assertFull();
assert (workerId == null);
if (workerId != null) {
String err = "Incorrect usage of createNewWorkerId(), current workerId is " + workerId + ", expecting null";
LOG.error(err);
throw new AssertionError(err);
}
synchronized (localState) {
workerId = Utils.uuid();
Map<String, Integer> workerToPort = localState.getApprovedWorkers();
Original file line number Diff line number Diff line change
@@ -111,9 +111,15 @@ protected Container(ContainerType type, Map<String, Object> conf, String supervi
int port, LocalAssignment assignment, ResourceIsolationInterface resourceIsolationManager,
String workerId, Map<String, Object> topoConf, AdvancedFSOps ops,
StormMetricsRegistry metricsRegistry, ContainerMemoryTracker containerMemoryTracker) throws IOException {
assert (type != null);
assert (conf != null);
assert (supervisorId != null);
if (type == null) {
throw new IOException("ContainerType parameter is null");
}
if (conf == null) {
throw new IOException("conf parameter value is null");
}
if (supervisorId == null) {
throw new IOException("SupervisorId parameter value is null");
}

symlinksDisabled = (boolean) conf.getOrDefault(Config.DISABLE_SYMLINKS, false);

@@ -137,14 +143,24 @@ protected Container(ContainerType type, Map<String, Object> conf, String supervi
}

if (this.type.isOnlyKillable()) {
assert (this.assignment == null);
assert (this.port <= 0);
assert (this.workerId != null);
if (this.assignment != null) {
throw new IOException("With ContainerType==OnlyKillable, expecting LocalAssignment member variable to be null");
}
if (this.port > 0) {
throw new IOException("With ContainerType==OnlyKillable, expecting port member variable <=0 but found " + this.port);
}
if (this.workerId == null) {
throw new IOException("With ContainerType==OnlyKillable, expecting WorkerId member variable to be assigned");
}
topologyId = null;
this.topoConf = null;
} else {
assert (assignment != null);
assert (port > 0);
if (this.assignment == null) {
throw new IOException("With ContainerType!=OnlyKillable, expecting LocalAssignment member variable to be assigned");
}
if (this.port <= 0) {
throw new IOException("With ContainerType!=OnlyKillable, expecting port member variable >0 but found " + this.port);
}
topologyId = assignment.get_topology_id();
if (!this.ops.doRequiredTopoFilesExist(this.conf, topologyId)) {
LOG.info(
@@ -175,7 +191,9 @@ public String toString() {
}

protected Map<String, Object> readTopoConf() throws IOException {
assert (topologyId != null);
if (topologyId == null) {
throw new IOException("Cannot readTopoConf: member variable topologyId is null");
}
return ConfigUtils.readSupervisorStormConf(conf, topologyId);
}

228 changes: 172 additions & 56 deletions storm-server/src/main/java/org/apache/storm/daemon/supervisor/Slot.java

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -193,14 +193,18 @@ LocallyCachedBlob getTopoConf(final String topologyId, String owner) {
}

private LocalizedResource getUserArchive(String user, String key) {
assert user != null : "All user archives require a user present";
if (user == null) {
throw new AssertionError("All user archives require a user present");
}
ConcurrentMap<String, LocalizedResource> keyToResource = userArchives.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
return keyToResource.computeIfAbsent(key,
(k) -> new LocalizedResource(key, localBaseDir, true, fsOps, conf, user, metricsRegistry));
}

private LocalizedResource getUserFile(String user, String key) {
assert user != null : "All user archives require a user present";
if (user == null) {
throw new AssertionError("All user archives require a user present");
}
ConcurrentMap<String, LocalizedResource> keyToResource = userFiles.computeIfAbsent(user, (u) -> new ConcurrentHashMap<>());
return keyToResource.computeIfAbsent(key,
(k) -> new LocalizedResource(key, localBaseDir, false, fsOps, conf, user, metricsRegistry));
Original file line number Diff line number Diff line change
@@ -147,7 +147,9 @@ public void assign(WorkerSlot slot, Collection<ExecutorDetails> executors) {
* Assign the slot to executors.
*/
public void assign(WorkerSlot slot, Collection<ExecutorDetails> executors, WorkerResources slotResources) {
assert slot != null;
if (slot == null) {
throw new AssertionError("WorkerSlot parameter is null");
}
for (ExecutorDetails executor : executors) {
this.executorToSlot.put(executor, slot);
}
Original file line number Diff line number Diff line change
@@ -556,8 +556,13 @@ private void initConfigs() {
this.topologyPriority =
ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_PRIORITY), null);

assert this.topologyWorkerMaxHeapSize != null;
assert this.topologyPriority != null;
// Fails in storm-core: org.apache.storm.scheduler-test / testname: test-cluster
//if (this.topologyWorkerMaxHeapSize == null) {
// throw new AssertionError("topologyWorkerMaxHeapSize is null");
//}
//if (this.topologyPriority == null) {
// throw new AssertionError("topologyPriority is null");
//}
}

/**
Original file line number Diff line number Diff line change
@@ -364,7 +364,9 @@ public void assignSingleExecutor(WorkerSlot ws, ExecutorDetails exec, TopologyDe
* @return true if it would fit else false
*/
public boolean wouldFit(WorkerSlot ws, ExecutorDetails exec, TopologyDetails td) {
assert nodeId.equals(ws.getNodeId()) : "Slot " + ws + " is not a part of this node " + nodeId;
if (!nodeId.equals(ws.getNodeId())) {
throw new AssertionError("Slot " + ws + " is not a part of this node " + nodeId);
}
if (!isAlive || !cluster.wouldFit(
ws,
exec,
Original file line number Diff line number Diff line change
@@ -127,8 +127,12 @@ protected boolean isExecAssignmentToWorkerValid(ExecutorDetails exec, WorkerSlot
*/
@VisibleForTesting
public static boolean validateSolution(Cluster cluster, TopologyDetails topo) {
assert (cluster.getAssignmentById(topo.getId()) != null);
LOG.debug("Checking for a valid scheduling for topology {}...", topo.getName());
if (cluster.getAssignmentById(topo.getId()) == null) {
String err = "cluster.getAssignmentById(\"" + topo.getId() + "\") returned null";
LOG.error(err);
throw new AssertionError("No assignments for topologyId " + topo.getId());
}

ConstraintSolverConfig constraintSolverConfig = new ConstraintSolverConfig(topo);

Original file line number Diff line number Diff line change
@@ -761,8 +761,8 @@ public void testTopologyWorkerMaxHeapSize() {
rs.schedule(topologies, cluster);
assertTrue(cluster.needsSchedulingRas(topology2));
String status = cluster.getStatusMap().get(topology2.getId());
assert status.startsWith("Not enough resources to schedule") : status;
//assert status.endsWith("5 executors not scheduled") : status;
String expectedStatusPrefix = "Not enough resources to schedule";
assertTrue(status.startsWith(expectedStatusPrefix), "Expected status to start with \"" + expectedStatusPrefix + "\" but status is: " + status);
assertEquals(5, cluster.getUnassignedExecutors(topology2).size());
} finally {
rs.cleanup();
@@ -1136,7 +1136,7 @@ void append(TimeBlockResult other) {

private long getMedianValue(List<Long> values) {
final int numValues = values.size();
assert(numValues % 2 == 1); // number of values must be odd to compute median as below
assertEquals(1, (numValues % 2), "Expecting odd number of values to compute median, got " + numValues);
List<Long> sortedValues = new ArrayList<>();
sortedValues.addAll(values);
Collections.sort(sortedValues);
Original file line number Diff line number Diff line change
@@ -48,6 +48,7 @@
import org.apache.storm.tuple.Tuple;
import org.apache.storm.tuple.Values;
import org.apache.storm.utils.Utils;
import org.junit.jupiter.api.Assertions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@@ -481,27 +482,25 @@ public static void assertTopologiesNotScheduled(Cluster cluster, Class strategyC
Topologies topologies = cluster.getTopologies();
for (String topoName : topoNames) {
TopologyDetails td = topologies.getByName(topoName);
String errMsg = "topology " + topoName + " using " + strategyClass.getName();
assert (td != null) : errMsg;
Assertions.assertNotNull(td, "Cannot find topology for topoName " + topoName);
String topoId = td.getId();
String status = cluster.getStatus(topoId);
assert (status != null) : errMsg;
assert (!isStatusSuccess(status)) : errMsg;
assert (cluster.getAssignmentById(topoId) == null) : errMsg;
assert (cluster.needsSchedulingRas(td)) : errMsg;
Assertions.assertNotNull(status, "Status unknown for topoName " + topoName);
Assertions.assertFalse(isStatusSuccess(status), "Successful status " + status + " for topoName " + topoName);
Assertions.assertNull(cluster.getAssignmentById(topoId), "Found assignment for topoId " + topoId);
Assertions.assertTrue(cluster.needsSchedulingRas(td), "Scheduling not required for topoName " + topoName);
}
}

public static void assertTopologiesFullyScheduled(Cluster cluster, Class strategyClass, String... topoNames) {
Topologies topologies = cluster.getTopologies();
for (String topoName : topoNames) {
TopologyDetails td = topologies.getByName(topoName);
String errMsg = "topology " + topoName + " using " + strategyClass.getName();
assert (td != null) : errMsg;
Assertions.assertNotNull(td, "Cannot find topology for topoName " + topoName);
String topoId = td.getId();
assertStatusSuccess(cluster, topoId);
assert (cluster.getAssignmentById(topoId) != null): errMsg;
assert (cluster.needsSchedulingRas(td) == false): errMsg;
Assertions.assertNotNull(cluster.getAssignmentById(topoId), "Cannot find assignment for topoId " + topoId);
Assertions.assertFalse(cluster.needsSchedulingRas(td), "Scheduling required for topoName " + topoName);
}
}

@@ -524,32 +523,31 @@ public static void assertTopologiesFullyScheduled(Cluster cluster, Class strateg
public static void assertTopologiesBeenEvicted(Cluster cluster, Class strategyClass, Set<String> evictedTopologies, String... topoNames) {
Topologies topologies = cluster.getTopologies();
LOG.info("Evicted topos: {}", evictedTopologies);
assert (evictedTopologies != null);
Assertions.assertNotNull(evictedTopologies, "evictedTopologies is null");
for (String topoName : topoNames) {
String errMsg = "topology " + topoName + " using " + strategyClass.getName();
TopologyDetails td = topologies.getByName(topoName);
assert (td != null) : errMsg;
Assertions.assertNotNull(td, "Cannot find topology for topoName " + topoName);
String topoId = td.getId();
assert (evictedTopologies.contains(topoId)) : errMsg;
Assertions.assertTrue(evictedTopologies.contains(topoId), "evictedTopologies does not contain topoId " + topoId);
}
}

public static void assertTopologiesNotBeenEvicted(Cluster cluster, Class strategyClass, Set<String> evictedTopologies, String... topoNames) {
Topologies topologies = cluster.getTopologies();
LOG.info("Evicted topos: {}", evictedTopologies);
assert (evictedTopologies != null);
Assertions.assertNotNull(evictedTopologies, "evictedTopologies is null");
for (String topoName : topoNames) {
String errMsg = "topology " + topoName + " using " + strategyClass.getName();
TopologyDetails td = topologies.getByName(topoName);
assert (td != null) : errMsg;
Assertions.assertNotNull(td, "Cannot find topology for topoName " + topoName);
String topoId = td.getId();
assert (!evictedTopologies.contains(topoId)) : errMsg;
Assertions.assertFalse(evictedTopologies.contains(topoId), "evictedTopologies contains topoId " + topoId);
}
}

public static void assertStatusSuccess(Cluster cluster, String topoId) {
assert (isStatusSuccess(cluster.getStatus(topoId))) :
"topology status " + topoId + " is not successful " + cluster.getStatus(topoId);
Assertions.assertTrue(isStatusSuccess(cluster.getStatus(topoId)), "topology " + topoId + " in unsuccessful status: " + cluster.getStatus(topoId));
}

public static boolean isStatusSuccess(String status) {
Original file line number Diff line number Diff line change
@@ -818,11 +818,11 @@ public void testMultipleRacks() {
assertEquals("rack-5", it.next().id, "Rack-5 should be ordered sixth");

SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
assert (schedulingResult.isSuccess());
SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-0
assertEquals("rack-0",
assertTrue(schedulingResult.isSuccess(), "Scheduling failed");
SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-0
assertEquals("rack-0",
resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-0");
}
assertEquals(0, cluster.getUnassignedExecutors(topo1).size(), "All executors in topo-1 scheduled");
@@ -842,11 +842,11 @@ public void testMultipleRacks() {
rs = new DefaultResourceAwareStrategyOld();
// schedule topo2
schedulingResult = rs.schedule(cluster, topo2);
assert (schedulingResult.isSuccess());
assignment = cluster.getAssignmentById(topo2.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-1
assertEquals("rack-1",
assertTrue(schedulingResult.isSuccess(), "Scheduling failed");
assignment = cluster.getAssignmentById(topo2.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-1
assertEquals("rack-1",
resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
}
assertEquals(0, cluster.getUnassignedExecutors(topo2).size(), "All executors in topo-2 scheduled");
@@ -944,12 +944,12 @@ public void testMultipleRacksWithFavoritism() {
assertEquals("rack-2", it.next().id, "rack-2 should be ordered fifth");

SchedulingResult schedulingResult = rs.schedule(cluster, topo1);
assert (schedulingResult.isSuccess());
SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
String hostName = rs.idToNode(ws.getNodeId()).getHostname();
String rackId = resolvedSuperVisors.get(hostName);
assertTrue(t1FavoredHostNames.contains(hostName) || "rack-0".equals(rackId),
assertTrue(schedulingResult.isSuccess(), "Scheduling failed");
SchedulerAssignment assignment = cluster.getAssignmentById(topo1.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
String hostName = rs.idToNode(ws.getNodeId()).getHostname();
String rackId = resolvedSuperVisors.get(hostName);
assertTrue(t1FavoredHostNames.contains(hostName) || "rack-0".equals(rackId),
ws + " is neither on a favored node " + t1FavoredHostNames + " nor the highest priority rack (rack-0)");
assertFalse(t1UnfavoredHostIds.contains(hostName),
ws + " is a part of an unfavored node " + t1UnfavoredHostIds);
@@ -971,13 +971,13 @@ public void testMultipleRacksWithFavoritism() {
rs = new DefaultResourceAwareStrategyOld();
// schedule topo2
schedulingResult = rs.schedule(cluster, topo2);
assert (schedulingResult.isSuccess());
assignment = cluster.getAssignmentById(topo2.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-1
// The favored nodes would have put it on a different rack, but because that rack does not have free space to run the
// topology it falls back to this rack
assertEquals("rack-1", resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
assertTrue(schedulingResult.isSuccess(), "Scheduling failed");
assignment = cluster.getAssignmentById(topo2.getId());
for (WorkerSlot ws : assignment.getSlotToExecutors().keySet()) {
//make sure all workers on scheduled in rack-1
// The favored nodes would have put it on a different rack, but because that rack does not have free space to run the
// topology it falls back to this rack
assertEquals("rack-1", resolvedSuperVisors.get(rs.idToNode(ws.getNodeId()).getHostname()), "assert worker scheduled on rack-1");
}
assertEquals(0, cluster.getUnassignedExecutors(topo2).size(), "All executors in topo-2 scheduled");
}
Original file line number Diff line number Diff line change
@@ -171,7 +171,9 @@ public static void verifyIncorrectJaasConf(ThriftServer server, Map<String, Obje
fail("An exception should have been thrown trying to connect.");
} catch (Exception e) {
LOG.info("Got Exception...", e);
assert (Utils.exceptionCauseIsInstanceOf(expectedException, e));
if (!Utils.exceptionCauseIsInstanceOf(expectedException, e)) {
throw new AssertionError("Expecting " + expectedException.getClass().getName() + " but got " + e.getClass().getName(), e);
}
}
}

@@ -245,9 +247,11 @@ public void simpleAuthTest() throws Exception {
try (NimbusClient client = new NimbusClient(badConf, "localhost", server.getPort(), NIMBUS_TIMEOUT)) {
client.getClient().activate("bad_security_auth_test_topology");
fail("An exception should have been thrown trying to connect.");
} catch (Exception te) {
LOG.info("Got Exception...", te);
assert (Utils.exceptionCauseIsInstanceOf(TTransportException.class, te));
} catch (Exception e) {
LOG.info("Got Exception...", e);
if (!Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) {
throw new AssertionError("Expecting TTransportException but got " + e.getClass().getName(), e);
}
}
});
verify(impl).activate("security_auth_test_topology");
@@ -279,9 +283,11 @@ public void digestAuthTest() throws Exception {
try (NimbusClient client = new NimbusClient(badTransport, "localhost", server.getPort(), NIMBUS_TIMEOUT)) {
client.getClient().activate("bad_security_auth_test_topology");
fail("An exception should have been thrown trying to connect.");
} catch (Exception te) {
LOG.info("Got Exception...", te);
assert (Utils.exceptionCauseIsInstanceOf(TTransportException.class, te));
} catch (Exception e) {
LOG.info("Got Exception...", e);
if (!Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) {
throw new AssertionError("Expecting TTransportException but got " + e.getClass().getName(), e);
}
}
//The user here from the jaas conf is bob. No impersonation is done, so verify that
ReqContext found = user.get();
@@ -327,7 +333,9 @@ public void workerTokenDigestAuthTest() throws Exception {
client.getClient().activate("bad_auth_test_topology");
fail("We should not be able to connect without a token...");
} catch (Exception e) {
assert (Utils.exceptionCauseIsInstanceOf(IOException.class, e));
if (!Utils.exceptionCauseIsInstanceOf(IOException.class, e)) {
throw new AssertionError("Expecting IOException but got " + e.getClass().getName(), e);
}
}

//Now let's create a token and verify that we can connect...
@@ -350,7 +358,9 @@ public void workerTokenDigestAuthTest() throws Exception {
tryConnectAs(conf, server, bob, "bad_auth_test_topology");
fail("We should not be able to connect with bad auth");
} catch (Exception e) {
assert (Utils.exceptionCauseIsInstanceOf(TTransportException.class, e));
if (!Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) {
throw new AssertionError("Expecting TTransportException but got " + e.getClass().getName(), e);
}
}
tryConnectAs(conf, server, alice, "topo-alice");
verifyUserIs(user, "alice");

0 comments on commit 2785507

Please sign in to comment.