Skip to content

Commit

Permalink
Merge pull request JanusGraph#469 from sjudeng/scan-fixes
Browse files Browse the repository at this point in the history
Resolve numerous high/medium Coverity findings
  • Loading branch information
sjudeng committed Sep 18, 2017
2 parents 00adcf5 + a164498 commit da87d78
Show file tree
Hide file tree
Showing 26 changed files with 150 additions and 46 deletions.
Expand Up @@ -437,6 +437,10 @@ public void clearStorage() throws BackendException {
if (this.storageConfig.get(GraphDatabaseConfiguration.DROP_ON_CLEAR)) {
ks.dropKeyspace();
} else {
final KeyspaceDefinition keyspaceDefinition = cluster.describeKeyspace(keySpaceName);
if (keyspaceDefinition == null) {
return;
}
for (final ColumnFamilyDefinition cf : cluster.describeKeyspace(keySpaceName).getColumnFamilyList()) {
ks.truncateColumnFamily(new ColumnFamily<Object, Object>(cf.getName(), null, null));
}
Expand Down
Expand Up @@ -28,6 +28,7 @@
import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration;

import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.*;
import static org.janusgraph.util.system.LoggerUtil.sanitizeAndLaunder;

import org.janusgraph.graphdb.database.StandardJanusGraph;

Expand Down Expand Up @@ -313,10 +314,10 @@ private static final String getAbsolutePath(final File configParent, String file
File storedir = new File(file);
if (!storedir.isAbsolute()) {
String newFile = configParent.getAbsolutePath() + File.separator + file;
log.debug("Overwrote relative path: was {}, now {}", file, newFile);
log.debug("Overwrote relative path: was {}, now {}", sanitizeAndLaunder(file), sanitizeAndLaunder(newFile));
return newFile;
} else {
log.debug("Loaded absolute path for key: {}", file);
log.debug("Loaded absolute path for key: {}", sanitizeAndLaunder(file));
return file;
}
}
Expand Down
Expand Up @@ -43,7 +43,7 @@ public IndexFeatures(boolean supportsDocumentTTL, Mapping defaultMap, ImmutableS
String wildcardField, ImmutableSet<Cardinality> supportedCardinaities, boolean supportsNanoseconds,
boolean supportCustomAnalyzer, boolean supportsGeoContains) {

Preconditions.checkArgument(defaultMap!=null || defaultMap!=Mapping.DEFAULT);
Preconditions.checkArgument(defaultMap!=null && defaultMap!=Mapping.DEFAULT);
Preconditions.checkArgument(supportedMap!=null && !supportedMap.isEmpty()
&& supportedMap.contains(defaultMap));
this.supportsDocumentTTL = supportsDocumentTTL;
Expand Down
Expand Up @@ -73,8 +73,7 @@ private void addJob(Object jobId, StandardScannerExecutor executor) {
runningJobs.remove(jobs.getKey(),exe);
}
}
runningJobs.putIfAbsent(jobId,executor);
Preconditions.checkArgument(runningJobs.get(jobId)==executor,"Another job with the same id is already running: %s",jobId);
Preconditions.checkArgument(runningJobs.putIfAbsent(jobId, executor) == null,"Another job with the same id is already running: %s",jobId);
}

public JanusGraphManagement.IndexJobFuture getRunningJob(Object jobId) {
Expand Down
Expand Up @@ -46,6 +46,7 @@
import org.janusgraph.graphdb.database.serialize.StandardSerializer;
import org.janusgraph.util.encoding.LongEncoding;
import org.janusgraph.util.system.ConfigurationUtil;
import org.janusgraph.util.system.LoggerUtil;
import org.janusgraph.util.system.NetworkUtil;

import org.apache.tinkerpop.gremlin.structure.Graph;
Expand Down Expand Up @@ -1391,10 +1392,12 @@ public GraphDatabaseConfiguration(ReadConfiguration localConfig) {
TimestampProviders backendPreference = null;
if (f.hasTimestamps() && null != (backendPreference = f.getPreferredTimestamps())) {
globalWrite.set(TIMESTAMP_PROVIDER, backendPreference);
log.info("Set timestamps to {} according to storage backend preference", globalWrite.get(TIMESTAMP_PROVIDER));
log.info("Set timestamps to {} according to storage backend preference",
LoggerUtil.sanitizeAndLaunder(globalWrite.get(TIMESTAMP_PROVIDER)));
}
globalWrite.set(TIMESTAMP_PROVIDER, TIMESTAMP_PROVIDER.getDefaultValue());
log.info("Set default timestamp provider {}", globalWrite.get(TIMESTAMP_PROVIDER));
log.info("Set default timestamp provider {}",
LoggerUtil.sanitizeAndLaunder(globalWrite.get(TIMESTAMP_PROVIDER)));
} else {
log.info("Using configured timestamp provider {}", localbc.get(TIMESTAMP_PROVIDER));
}
Expand Down
Expand Up @@ -232,7 +232,7 @@ public Entry writeRelation(InternalRelation relation, int position, TypeInspecto
}

public StaticArrayEntry writeRelation(InternalRelation relation, InternalRelationType type, int position, TypeInspector tx) {
assert type==relation.getType() || type.getBaseType().equals(relation.getType());
assert type==relation.getType() || (type.getBaseType() != null && type.getBaseType().equals(relation.getType()));
Direction dir = EdgeDirection.fromPosition(position);
Preconditions.checkArgument(type.isUnidirected(Direction.BOTH) || type.isUnidirected(dir));
long typeid = type.longId();
Expand Down
Expand Up @@ -669,6 +669,7 @@ public void commit(final Collection<InternalRelation> addedRelations,
if (logTransaction) {
//[FAILURE] Inability to log transaction fails the transaction by escalation since it's likely due to unavailability of primary
//storage backend.
Preconditions.checkState(txLog != null, "Transaction log is null");
txLog.add(txLogHeader.serializeModifications(serializer, LogTxStatus.PRECOMMIT, tx, addedRelations, deletedRelations),txLogHeader.getLogKey());
}

Expand Down
Expand Up @@ -366,6 +366,9 @@ public void run() {
try {
for (Map.Entry<Long, Map<String, Object>> vprop : properties) {
Vertex v = tx.getVertex(vprop.getKey());
if (v == null) {
continue;
}
for (Map.Entry<String, Object> prop : vprop.getValue().entrySet()) {
v.property(VertexProperty.Cardinality.single, prop.getKey(), prop.getValue());
}
Expand Down
Expand Up @@ -16,6 +16,7 @@

import com.google.common.base.Preconditions;
import org.janusgraph.core.JanusGraphEdge;
import org.janusgraph.core.JanusGraphException;
import org.janusgraph.core.JanusGraphVertex;
import org.janusgraph.core.JanusGraphVertexProperty;
import org.janusgraph.graphdb.vertices.PreloadedVertex;
Expand Down Expand Up @@ -117,10 +118,15 @@ public Stream<M> receiveMessages(MessageScope messageScope) {
else return Stream.of(message);
} else {
final MessageScope.Local<M> localMessageScope = (MessageScope.Local) messageScope;
final Traversal<Vertex, Edge> reverseIncident = FulgoraUtil.getReverseElementTraversal(localMessageScope,vertex,vertex.tx());
final BiFunction<M,Edge,M> edgeFct = localMessageScope.getEdgeFunction();
final List<Edge> edges;
try (final Traversal<Vertex, Edge> reverseIncident = FulgoraUtil.getReverseElementTraversal(localMessageScope,vertex,vertex.tx())) {
edges = IteratorUtils.list(reverseIncident);
} catch (Exception e) {
throw new JanusGraphException("Unable to close traversal", e);
}

return IteratorUtils.stream(reverseIncident)
return edges.stream()
.map(e -> {
M msg = vertexMemory.getMessage(vertexMemory.getCanonicalId(((JanusGraphEdge) e).otherVertex(vertex).longId()), localMessageScope);
return msg == null ? null : edgeFct.apply(msg, e);
Expand Down
Expand Up @@ -87,15 +87,12 @@ private <T> T runWithMetrics(String opName, Function<Void,T> impl) {

final MetricManager mgr = MetricManager.INSTANCE;
mgr.getCounter(metricsPrefix, opName, M_CALLS).inc();
final Timer.Context tc = mgr.getTimer(metricsPrefix, opName, M_TIME).time();

try {
try (final Timer.Context tc = mgr.getTimer(metricsPrefix, opName, M_TIME).time()) {
return impl.apply(null);
} catch (RuntimeException e) {
mgr.getCounter(metricsPrefix, opName, M_EXCEPTIONS).inc();
throw e;
} finally {
tc.stop();
}
}
}
Expand Up @@ -269,7 +269,7 @@ public static <R> List<R> processIntersectingRetrievals(List<IndexCall<R>> retri
}
}
sublimit = (int) Math.min(Integer.MAX_VALUE - 1, Math.max(Math.pow(sublimit, 1.5),(sublimit+1)*2));
} while (results.size() < limit && !exhaustedResults);
} while (results != null && results.size() < limit && !exhaustedResults);
return results;
}

Expand Down
Expand Up @@ -344,7 +344,7 @@ public VertexList executeVertexIds(InternalVertex vertex, BaseVertexCentricQuery
if (merge==null) merge = (VertexListInternal)vlist;
else merge.addAll(vlist);
}
if (merge.size()>baseQuery.getLimit()) merge = (VertexListInternal)merge.subList(0,baseQuery.getLimit());
if (merge != null && merge.size()>baseQuery.getLimit()) merge = (VertexListInternal)merge.subList(0,baseQuery.getLimit());
return merge;
} else vertex = tx.getCanonicalVertex(vertex);
}
Expand Down
Expand Up @@ -15,6 +15,7 @@
package org.janusgraph.graphdb.relations;

import com.carrotsearch.hppc.cursors.LongObjectCursor;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import org.janusgraph.core.PropertyKey;
Expand Down Expand Up @@ -46,7 +47,9 @@ public CacheEdge(long id, EdgeLabel label, InternalVertex start, InternalVertex
}

public Direction getVertexCentricDirection() {
return data.getCache().direction;
final RelationCache cache = data.getCache();
Preconditions.checkState(cache != null, "Cache is null");
return cache.direction;
}

//############## Similar code as CacheProperty but be careful when copying #############################
Expand Down
Expand Up @@ -163,8 +163,7 @@ public boolean equals(Object other) {
else if (other==null) return false;
else if (!getClass().isInstance(other)) return false;
RangeInterval oth = (RangeInterval)other;
if ((start==null ^ oth.start==null) || (end==null ^ oth.end==null)) return false;
return start.equals(oth.start) && end.equals(oth.end) && endInclusive==oth.endInclusive && startInclusive==oth.startInclusive;
return Objects.equals(start, oth.start) && Objects.equals(end, oth.end) && end.equals(oth.end) && endInclusive==oth.endInclusive && startInclusive==oth.startInclusive;
}

@Override
Expand Down
Expand Up @@ -64,4 +64,11 @@ public static final int getAsciiByteLength(String attribute) {
return attribute.isEmpty()?1:attribute.length();
}

public static String launder(String input) {
Preconditions.checkNotNull(input);
final StringBuilder sb = new StringBuilder();
input.chars().forEach(c -> sb.append((char) Integer.valueOf(c).intValue()));
return sb.toString();
}

}
Expand Up @@ -40,7 +40,7 @@ public static void main(String args[]) throws IOException {
System.exit(1);
}

log.info("Checking " + args[0]);
log.info("Checking " + LoggerUtil.sanitizeAndLaunder(args[0]));
Status s = validate(args[0]);
if (0 == s.errors) {
log.info(s.toString());
Expand Down
@@ -0,0 +1,30 @@
// Copyright 2017 JanusGraph Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package org.janusgraph.util.system;

import org.janusgraph.util.encoding.StringEncoding;

public class LoggerUtil {

/**
* Encode newline characters and launder string to mitigate log forging vulnerability.
* @param input
* @return
*/
public static String sanitizeAndLaunder(Object input) {
return input != null ? StringEncoding.launder(input.toString().replaceAll("[\n\r]", "%0A")) : null;
}

}
Expand Up @@ -67,7 +67,7 @@ protected StaticBuffer computeNext() {

while (this.iterator.hasNext()) {
this.iterator.next();
if (!this.currentKey.equals(this.lastKey)) {
if (this.currentKey != null && !this.currentKey.equals(this.lastKey)) {
this.lastKey = this.currentKey;
return this.lastKey;
}
Expand Down
Expand Up @@ -201,7 +201,7 @@ public Map getIndexSettings(String indexName) throws IOException {
Response response = performRequest(REQUEST_TYPE_GET, REQUEST_SEPARATOR + indexName + "/_settings", null);
try (final InputStream inputStream = response.getEntity().getContent()) {
Map<String,RestIndexSettings> settings = mapper.readValue(inputStream, new TypeReference<Map<String, RestIndexSettings>>() {});
return settings.get(indexName).getSettings().getMap();
return settings == null ? null : settings.get(indexName).getSettings().getMap();
}
}

Expand All @@ -215,7 +215,7 @@ public Map getMapping(String indexName, String typeName) throws IOException{
Response response = performRequest(REQUEST_TYPE_GET, REQUEST_SEPARATOR + indexName + "/_mapping/" + typeName, null);
try (final InputStream inputStream = response.getEntity().getContent()) {
Map<String, RestIndexMappings> settings = mapper.readValue(inputStream, new TypeReference<Map<String, RestIndexMappings>>() {});
return settings.get(indexName).getMappings().get(typeName).getProperties();
return settings == null ? null : settings.get(indexName).getMappings().get(typeName).getProperties();
}
}

Expand Down
Expand Up @@ -17,7 +17,9 @@
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import org.apache.commons.lang3.StringUtils;
import org.janusgraph.core.JanusGraph;
import org.janusgraph.core.RelationType;
import org.janusgraph.core.schema.RelationTypeIndex;
import org.janusgraph.core.schema.SchemaAction;
import org.janusgraph.core.schema.JanusGraphIndex;
Expand Down Expand Up @@ -163,10 +165,9 @@ public JanusGraphManagement.IndexJobFuture updateIndex(Index index, SchemaAction

// The index name and relation type name (if the latter is applicable)
final String indexName = index.name();
final String relationTypeName =
RelationTypeIndex.class.isAssignableFrom(index.getClass()) ?
((RelationTypeIndex)index).getType().name() :
"";
final RelationType relationType =
RelationTypeIndex.class.isAssignableFrom(index.getClass()) ? ((RelationTypeIndex) index).getType() : null;
final String relationTypeName = relationType == null ? StringUtils.EMPTY : relationType.name();
Preconditions.checkNotNull(indexName);

// Set the class of the IndexUpdateJob
Expand Down
Expand Up @@ -99,7 +99,7 @@ protected Path uploadFileIfNecessary(FileSystem localFS, Path localPath, FileSys
localPath, destPath, destFS);
}

if (!fileStats.isRemoteCopyCurrent()) {
if (fileStats != null && !fileStats.isRemoteCopyCurrent()) {
log.debug("Copying {} to {}", localPath, destPath);
destFS.copyFromLocalFile(localPath, destPath);
if (null != fileStats.local) {
Expand Down Expand Up @@ -146,8 +146,9 @@ private static Conf configureByClasspath(String mapredJarFilename) {
for (String cpentry : classpath.split(File.pathSeparator)) {
if (cpentry.toLowerCase().endsWith(".jar") || cpentry.toLowerCase().endsWith(".properties")) {
paths.add(new Path(cpentry));
if (cpentry.toLowerCase().endsWith(mrj));
if (cpentry.toLowerCase().endsWith(mrj)) {
mapredJarPath = cpentry;
}
}
}
return new Conf(paths, mapredJarPath);
Expand Down
Expand Up @@ -103,9 +103,11 @@ public static ScanMetrics runJob(Configuration conf, String confRootField,
JanusGraphHadoopConfiguration.SCAN_JOB_CONFIG_KEYS, scanConf);

// Copy scanjob settings from the JanusGraph Configuration instance to the Hadoop Configuration instance
Map<String, Object> jobConfMap = conf.getSubset(confRoot);
for (Map.Entry<String, Object> jobConfEntry : jobConfMap.entrySet()) {
hadoopJobConf.set((ConfigOption) ConfigElement.parse(confRoot, jobConfEntry.getKey()).element, jobConfEntry.getValue());
if (conf != null) {
Map<String, Object> jobConfMap = conf.getSubset(confRoot);
for (Map.Entry<String, Object> jobConfEntry : jobConfMap.entrySet()) {
hadoopJobConf.set((ConfigOption) ConfigElement.parse(confRoot, jobConfEntry.getKey()).element, jobConfEntry.getValue());
}
}
}

Expand Down
Expand Up @@ -407,7 +407,7 @@ private void addToDocument(String store,
for (IndexableField f : spatialStrategy.createIndexableFields(geo.getValue())) {
doc.add(f);
if (spatialStrategy instanceof PointVectorStrategy) {
doc.add(new DoubleDocValuesField(f.name(), f.numericValue().doubleValue()));
doc.add(new DoubleDocValuesField(f.name(), f.numericValue() == null ? null : f.numericValue().doubleValue()));
}
}
}
Expand Down Expand Up @@ -453,7 +453,8 @@ public Stream<String> query(IndexQuery query, KeyInformation.IndexRetriever info
log.debug("Executed query [{}] in {} ms", q, System.currentTimeMillis() - time);
List<String> result = new ArrayList<String>(docs.scoreDocs.length);
for (int i = 0; i < docs.scoreDocs.length; i++) {
result.add(searcher.doc(docs.scoreDocs[i].doc).getField(DOCID).stringValue());
final IndexableField field = searcher.doc(docs.scoreDocs[i].doc).getField(DOCID);
result.add(field == null ? null : field.stringValue());
}
return result.stream();
} catch (IOException e) {
Expand Down Expand Up @@ -642,7 +643,8 @@ public Stream<RawQuery.Result<String>> query(RawQuery query, KeyInformation.Inde
log.debug("Executed query [{}] in {} ms",q, System.currentTimeMillis() - time);
List<RawQuery.Result<String>> result = new ArrayList<RawQuery.Result<String>>(docs.scoreDocs.length);
for (int i = offset; i < docs.scoreDocs.length; i++) {
result.add(new RawQuery.Result<String>(searcher.doc(docs.scoreDocs[i].doc).getField(DOCID).stringValue(),docs.scoreDocs[i].score));
final IndexableField field = searcher.doc(docs.scoreDocs[i].doc).getField(DOCID);
result.add(new RawQuery.Result<String>(field == null ? null : field.stringValue(),docs.scoreDocs[i].score));
}
return result.stream();
} catch (IOException e) {
Expand Down

0 comments on commit da87d78

Please sign in to comment.