Skip to content

Commit

Permalink
Fixes to some of the issues reported by fbinfer static code analyzer (a…
Browse files Browse the repository at this point in the history
  • Loading branch information
saandrews authored and merlimat committed Apr 12, 2017
1 parent f572241 commit 8bc4235
Show file tree
Hide file tree
Showing 9 changed files with 22 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,7 @@ private void siftDown(final Item item) {
Item right = getRight(item);
if (right != null && right.position.compareTo(item.position) < 0) {
Item left = getLeft(item);
if (left.position.compareTo(right.position) < 0) {
if (left != null && left.position.compareTo(right.position) < 0) {
j = left;
} else {
j = right;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1319,6 +1319,11 @@ public void doLoadShedding() {
if (isAboveLoadLevel(lr.getSystemResourceUsage(), overloadThreshold)) {
ResourceType bottleneckResourceType = lr.getBottleneckResourceType();
Map<String, NamespaceBundleStats> bundleStats = lr.getSortedBundleStats(bottleneckResourceType);
if (bundleStats == null) {
log.warn("Null bundle stats for bundle {}", lr.getName());
continue;

}
// 1. owns only one namespace
if (bundleStats.size() == 1) {
// can't unload one namespace, just issue a warning message
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,11 +81,13 @@ private NamespaceIsolationPolicy getNamespaceIsolationPolicy(NamespaceName names
}

public boolean isPrimaryBroker(NamespaceName namespace, String broker) {
return getNamespaceIsolationPolicy(namespace).isPrimaryBroker(broker);
NamespaceIsolationPolicy nsPolicy = getNamespaceIsolationPolicy(namespace);
return (nsPolicy != null) ? nsPolicy.isPrimaryBroker(broker) : false;
}

public boolean isSecondaryBroker(NamespaceName namespace, String broker) {
return getNamespaceIsolationPolicy(namespace).isSecondaryBroker(broker);
NamespaceIsolationPolicy nsPolicy = getNamespaceIsolationPolicy(namespace);
return (nsPolicy != null) ? nsPolicy.isSecondaryBroker(broker) : false;
}

public boolean isSharedBroker(String broker) {
Expand All @@ -101,6 +103,7 @@ public boolean isSharedBroker(String broker) {
}

public boolean shouldFailoverToSecondaries(NamespaceName namespace, int totalPrimaryCandidates) {
return getNamespaceIsolationPolicy(namespace).shouldFailover(totalPrimaryCandidates);
NamespaceIsolationPolicy nsPolicy = getNamespaceIsolationPolicy(namespace);
return (nsPolicy != null) ? nsPolicy.shouldFailover(totalPrimaryCandidates) : false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@

import com.google.common.collect.Lists;
import com.google.common.hash.Hashing;
import com.yahoo.pulsar.broker.LocalBrokerData;
import com.yahoo.pulsar.broker.PulsarServerException;
import com.yahoo.pulsar.broker.PulsarService;
import com.yahoo.pulsar.broker.ServiceConfiguration;
Expand All @@ -67,12 +66,10 @@
import com.yahoo.pulsar.common.policies.data.BundlesData;
import com.yahoo.pulsar.common.policies.data.LocalPolicies;
import com.yahoo.pulsar.common.policies.data.NamespaceOwnershipStatus;
import com.yahoo.pulsar.common.policies.data.loadbalancer.LoadReport;
import com.yahoo.pulsar.common.policies.data.loadbalancer.ServiceLookupData;
import com.yahoo.pulsar.common.policies.impl.NamespaceIsolationPolicies;
import com.yahoo.pulsar.common.util.Codec;
import com.yahoo.pulsar.common.util.ObjectMapperFactory;
import com.yahoo.pulsar.zookeeper.ZooKeeperCache.Deserializer;

/**
* The <code>NamespaceService</code> provides resource ownership lookup as well as resource ownership claiming services
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import com.yahoo.pulsar.client.api.AuthenticationFactory;
import com.yahoo.pulsar.client.api.ClientConfiguration;
import com.yahoo.pulsar.client.api.PulsarClientException;
import com.yahoo.pulsar.client.impl.auth.AuthenticationDisabled;
import com.yahoo.pulsar.common.util.SecurityUtility;

/**
Expand Down Expand Up @@ -98,7 +99,7 @@ public class PulsarAdmin implements Closeable {
* the ClientConfiguration object to be used to talk with Pulsar
*/
public PulsarAdmin(URL serviceUrl, ClientConfiguration pulsarConfig) throws PulsarClientException {
this.auth = pulsarConfig != null ? pulsarConfig.getAuthentication() : null;
this.auth = pulsarConfig != null ? pulsarConfig.getAuthentication() : new AuthenticationDisabled();
LOG.debug("created: serviceUrl={}, authMethodName={}", serviceUrl,
auth != null ? auth.getAuthMethodName() : null);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ private List<byte[]> generateMessageBodies(List<String> stringMessages, List<Str
byte[] fileBytes = new byte[(int) f.length()];
fis.read(fileBytes);
messageBodies.add(fileBytes);
fis.close();

}
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,9 @@ void reset() {
}

void updateCumulativeStats(ConsumerStats stats) {
if (stats == null) {
return;
}
numMsgsReceived.add(stats.numMsgsReceived.longValue());
numBytesReceived.add(stats.numBytesReceived.longValue());
numReceiveFailed.add(stats.numReceiveFailed.longValue());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,9 @@ void reset() {
}

void updateCumulativeStats(ProducerStats stats) {
if (stats == null) {
return;
}
numMsgsSent.add(stats.numMsgsSent.longValue());
numBytesSent.add(stats.numBytesSent.longValue());
numSendFailed.add(stats.numSendFailed.longValue());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,7 @@ public void assignBroker(NamespaceName nsname, BrokerStatus brkStatus, SortedSet
BrokerAssignment brokerAssignment = this.getBrokerAssignment(nsPolicy, brkStatus.getBrokerAddress());
if (brokerAssignment == BrokerAssignment.primary) {
// Only add to candidates if allowed by policy
if (nsPolicy.isPrimaryBrokerAvailable(brkStatus)) {
if (nsPolicy != null && nsPolicy.isPrimaryBrokerAvailable(brkStatus)) {
primaryCandidates.add(brkStatus);
}
} else if (brokerAssignment == BrokerAssignment.secondary) {
Expand Down

0 comments on commit 8bc4235

Please sign in to comment.