Enable most IntelliJ 'Probable bugs' inspections (#4353)

* Enable most IntelliJ 'Probable bugs' inspections

* Fix in RemoteTestNG

* Fix IndexSpec's equals() and hashCode() to include longEncoding

* Fix inspection errors

* Extract global isntance of natural().nullsFirst(); address comments

* Fix

* Use noinspection comments instead of SuppressWarnings on method for IntelliJ-specific inspections

* Prohibit Ordering.natural().nullsFirst() using Checkstyle
This commit is contained in:
Roman Leventov 2017-06-07 11:54:25 -05:00 committed by Charles Allen
parent b487fa355b
commit 63a897c278
56 changed files with 354 additions and 175 deletions

View File

@ -5,18 +5,103 @@
<inspection_tool class="AntDuplicateTargetsInspection" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="AntMissingPropertiesFileInspection" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="AntResolveInspection" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ArrayEquality" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ArrayEquals" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ArrayHashCode" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ArrayObjectsEquals" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="AssertWithSideEffects" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="CastConflictsWithInstanceof" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="CastToIncompatibleInterface" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="CheckValidXmlInScriptTagBody" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ClassNewInstance" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="CollectionAddedToSelf" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ComparableImplementedButEqualsNotOverridden" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ComparatorMethodParameterNotUsed" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="CompareToUsesNonFinalVariable" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ConstantAssertCondition" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="Contract" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="CovariantEquals" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="EmptyInitializer" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="EmptyStatementBody" enabled="true" level="WARNING" enabled_by_default="true">
<option name="m_reportEmptyBlocks" value="true" />
<option name="commentsAreContent" value="true" />
</inspection_tool>
<inspection_tool class="EqualsAndHashcode" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="EqualsBetweenInconvertibleTypes" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="EqualsUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="EqualsWithItself" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ForLoopThatDoesntUseLoopVariable" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="HashCodeUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ImplicitArrayToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InfiniteRecursion" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InjectedReferences" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="InnerClassReferencedViaSubclass" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InstanceofIncompatibleInterface" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InstantiationOfUtilityClass" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InvalidComparatorMethodReference" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IteratorHasNextCallsIteratorNext" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IteratorNextDoesNotThrowNoSuchElementException" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="JavadocReference" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="JsonStandardCompliance" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="MalformedRegex" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="MathRandomCastToInt" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="MavenModelInspection" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="MismatchedArrayReadWrite" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="MismatchedStringBuilderQueryUpdate" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="MisspelledEquals" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="NewStringBufferWithCharArgument" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="NonShortCircuitBoolean" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="NullArgumentToVariableArgMethod" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="NumberEquality" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ObjectEquality" enabled="true" level="WARNING" enabled_by_default="true">
<option name="m_ignoreEnums" value="true" />
<option name="m_ignoreClassObjects" value="false" />
<option name="m_ignorePrivateConstructors" value="false" />
</inspection_tool>
<inspection_tool class="ObjectEqualsNull" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ObjectToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="PrimitiveArrayArgumentToVariableArgMethod" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ReflectionForUnavailableAnnotation" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ReplaceAllDot" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ResultOfObjectAllocationIgnored" enabled="true" level="WARNING" enabled_by_default="true">
<scope name="Production" level="ERROR" enabled="true" />
</inspection_tool>
<inspection_tool class="ResultSetIndexZero" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ReturnNull" enabled="true" level="WARNING" enabled_by_default="true">
<option name="m_reportObjectMethods" value="true" />
<option name="m_reportArrayMethods" value="true" />
<option name="m_reportCollectionMethods" value="true" />
<option name="m_ignorePrivateMethods" value="false" />
</inspection_tool>
<inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">
<option name="processCode" value="true" />
<option name="processLiterals" value="true" />
<option name="processComments" value="true" />
</inspection_tool>
<inspection_tool class="StaticCallOnSubclass" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="StaticFieldReferenceOnSubclass" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="StringConcatenationInFormatCall" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="StringConcatenationInMessageFormatCall" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="StringConcatenationMissingWhitespace" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="StringEquality" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="StringTokenizerDelimiter" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SubtractionInCompareTo" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="SuspiciousArrayCast" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="SuspiciousIndentAfterControlStatement" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SuspiciousMethodCalls" enabled="true" level="ERROR" enabled_by_default="true">
<option name="REPORT_CONVERTIBLE_METHOD_CALLS" value="true" />
</inspection_tool>
<inspection_tool class="SuspiciousNameCombination" enabled="true" level="ERROR" enabled_by_default="true">
<group names="x,width,left,right" />
<group names="y,height,top,bottom" />
</inspection_tool>
<inspection_tool class="SuspiciousSystemArraycopy" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SuspiciousToArrayCall" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SyntaxError" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="TextLabelInSwitchStatement" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ThrowableNotThrown" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="UseOfPropertiesAsHashtable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="VariableNotUsedInsideIf" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="XmlHighlighting" enabled="true" level="WARNING" enabled_by_default="true" />
</profile>
</component>

View File

@ -47,12 +47,13 @@ public class TimestampSpec
private final String timestampColumn;
private final String timestampFormat;
private final Function<Object, DateTime> timestampConverter;
// this value should never be set for production data
private final DateTime missingValue;
/** This field is a derivative of {@link #timestampFormat}; not checked in {@link #equals} and {@link #hashCode} */
private final Function<Object, DateTime> timestampConverter;
// remember last value parsed
private ParseCtx parseCtx = new ParseCtx();
private transient ParseCtx parseCtx = new ParseCtx();
@JsonCreator
public TimestampSpec(
@ -141,6 +142,16 @@ public class TimestampSpec
return result;
}
@Override
public String toString()
{
return "TimestampSpec{" +
"timestampColumn='" + timestampColumn + '\'' +
", timestampFormat='" + timestampFormat + '\'' +
", missingValue=" + missingValue +
'}';
}
//simple merge strategy on timestampSpec that checks if all are equal or else
//returns null. this can be improved in future but is good enough for most use-cases.
public static TimestampSpec mergeTimestampSpec(List<TimestampSpec> toMerge) {

View File

@ -29,10 +29,18 @@
<property name="fileExtensions" value="java"/>
</module>
<module name="SuppressionCommentFilter">
<property name="offCommentFormat" value="CHECKSTYLE.OFF\: ([\w\|]+)"/>
<property name="onCommentFormat" value="CHECKSTYLE.ON\: ([\w\|]+)"/>
<property name="checkFormat" value="$1"/>
</module>
<module name="NewlineAtEndOfFile"/>
<module name="FileTabCharacter"/>
<module name="TreeWalker">
<module name="FileContentsHolder"/>
<!-- See http://checkstyle.sourceforge.net/checks.html for examples -->
<!--<module name="LineLength">-->
@ -44,19 +52,24 @@
<module name="UnusedImports" />
<module name="NeedBraces"/>
<module name="Regexp">
<!-- Prohibit using Guava's Closer, see comment in io.druid.java.util.common.io.Closer -->
<property name="format" value="com\.google\.common\.io\.Closer"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use io.druid.java.util.common.io.Closer instead of Guava's Closer"/>
</module>
<module name="Regexp">
<!-- Prohibit IntelliJ-style commented code lines from being committed -->
<property name="format" value="^// {2}"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Don't commit IntelliJ-style commented code lines"/>
</module>
<module name="Regexp">
<!-- Force comments to classes and methods to be Javadoc comments -->
<property name="format" value="/\*[^\*].*?\n(\s*\*.*?\n)*\s+\*/[\s\n]*(transient|volatile|strictfp|synchronized|native|abstract|class|interface|enum|static|private|public|protected|default|void|byte|char|short|int|float|long|double|[A-Z])"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Comments to classes and methods must be Javadoc comments"/>
</module>
<module name="Regexp">
<property name="format" value="natural\(\)[\s\n]*\.[\s\n]*nullsFirst\(\)"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Comparators.naturalNullsFirst() instead of Ordering.natural().nullsFirst()"/>
</module>
</module>
</module>

View File

@ -19,16 +19,10 @@
package io.druid.timeline.partition;
import com.google.common.collect.Ordering;
import java.util.Comparator;
/**
*/
public class IntegerPartitionChunk<T> implements PartitionChunk<T>
{
Comparator<Integer> comparator = Ordering.<Integer>natural().nullsFirst();
private final Integer start;
private final Integer end;
private final int chunkNumber;
@ -96,7 +90,7 @@ public class IntegerPartitionChunk<T> implements PartitionChunk<T>
{
if (chunk instanceof IntegerPartitionChunk) {
IntegerPartitionChunk<T> intChunk = (IntegerPartitionChunk<T>) chunk;
return comparator.compare(chunkNumber, intChunk.chunkNumber);
return Integer.compare(chunkNumber, intChunk.chunkNumber);
} else {
throw new IllegalArgumentException("Cannot compare against something that is not an IntegerPartitionChunk.");
}

View File

@ -19,14 +19,8 @@
package io.druid.timeline.partition;
import com.google.common.collect.Ordering;
import java.util.Comparator;
public class LinearPartitionChunk <T> implements PartitionChunk<T>
{
Comparator<Integer> comparator = Ordering.<Integer>natural().nullsFirst();
private final int chunkNumber;
private final T object;
@ -81,7 +75,7 @@ public class LinearPartitionChunk <T> implements PartitionChunk<T>
if (chunk instanceof LinearPartitionChunk) {
LinearPartitionChunk<T> linearChunk = (LinearPartitionChunk<T>) chunk;
return comparator.compare(chunkNumber, linearChunk.chunkNumber);
return Integer.compare(chunkNumber, linearChunk.chunkNumber);
}
throw new IllegalArgumentException("Cannot compare against something that is not a LinearPartitionChunk.");
}

View File

@ -19,16 +19,10 @@
package io.druid.timeline.partition;
import com.google.common.collect.Ordering;
import java.util.Comparator;
/**
*/
public class StringPartitionChunk<T> implements PartitionChunk<T>
{
private static final Comparator<Integer> comparator = Ordering.<Integer>natural().nullsFirst();
private final String start;
private final String end;
private final int chunkNumber;
@ -95,7 +89,7 @@ public class StringPartitionChunk<T> implements PartitionChunk<T>
if (chunk instanceof StringPartitionChunk) {
StringPartitionChunk<T> stringChunk = (StringPartitionChunk<T>) chunk;
return comparator.compare(chunkNumber, stringChunk.chunkNumber);
return Integer.compare(chunkNumber, stringChunk.chunkNumber);
}
throw new IllegalArgumentException("Cannot compare against something that is not a StringPartitionChunk.");
}

View File

@ -19,15 +19,13 @@
package io.druid.common.guava;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.guava.Yielder;
import io.druid.java.util.common.guava.YieldingAccumulator;
import io.druid.java.util.common.guava.nary.BinaryFn;
import org.junit.Assert;
import org.junit.Test;
@ -82,7 +80,7 @@ public class ComplexSequenceTest
private Sequence<Integer> combine(Sequence<Integer> sequence)
{
return CombiningSequence.create(sequence, alwaysSame, plus);
return CombiningSequence.create(sequence, Comparators.alwaysEqual(), plus);
}
private Sequence<Integer> concat(Sequence<Integer>... sequences)
@ -90,15 +88,6 @@ public class ComplexSequenceTest
return Sequences.concat(Arrays.asList(sequences));
}
private final Ordering<Integer> alwaysSame = new Ordering<Integer>()
{
@Override
public int compare(Integer left, Integer right)
{
return 0;
}
};
private final BinaryFn<Integer, Integer, Integer> plus = new BinaryFn<Integer, Integer, Integer>()
{
@Override

View File

@ -468,7 +468,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
* string. The bit <i>must</i> be appendable, that is it must represent an
* integer that is strictly greater than the maximum integer in the set.
* Note that the parameter range check is performed by the public method
* {@link #add(Integer)} and <i>not</i> in this method.
* {@link #add)} and <i>not</i> in this method.
* <p/>
* <b>NOTE:</b> This method assumes that the last element of {@link #words}
* (i.e. <code>getLastWord()</code>) <i>must</i> be one of the
@ -712,6 +712,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
if (!otherItr.isLiteral) {
int minCount = Math.min(thisItr.count, otherItr.count);
res.appendFill(minCount, operator.combineLiterals(thisItr.word, otherItr.word));
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(minCount) | !otherItr.prepareNext(minCount)) // NOT ||
{
break;
@ -719,6 +720,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
} else {
res.appendLiteral(operator.combineLiterals(thisItr.toLiteral(), otherItr.word));
thisItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(1) | !otherItr.prepareNext()) // do NOT use "||"
{
break;
@ -727,12 +729,14 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
} else if (!otherItr.isLiteral) {
res.appendLiteral(operator.combineLiterals(thisItr.word, otherItr.toLiteral()));
otherItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext(1)) // do NOT use "||"
{
break;
}
} else {
res.appendLiteral(operator.combineLiterals(thisItr.word, otherItr.word));
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext()) // do NOT use "||"
{
break;
@ -791,6 +795,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
/**
* {@inheritDoc}
*/
@SuppressWarnings("NonShortCircuitBooleanExpression")
@Override
public int intersectionSize(IntSet o)
{
@ -1715,6 +1720,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
if ((ConciseSetUtils.SEQUENCE_BIT & thisItr.word & otherItr.word) != 0) {
return true;
}
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(minCount) | !otherItr.prepareNext(minCount)) // NOT ||
{
return false;
@ -1724,6 +1730,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
return true;
}
thisItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(1) | !otherItr.prepareNext()) // do NOT use "||"
{
return false;
@ -1734,6 +1741,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
return true;
}
otherItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext(1)) // do NOT use "||"
{
return false;
@ -1742,6 +1750,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
if ((thisItr.word & otherItr.word) != ConciseSetUtils.ALL_ZEROS_LITERAL) {
return true;
}
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext()) // do NOT use "||"
{
return false;
@ -1804,6 +1813,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
return true;
}
}
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(minCount) | !otherItr.prepareNext(minCount)) // NOT ||
{
return false;
@ -1814,6 +1824,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
return true;
}
thisItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext(1) | !otherItr.prepareNext()) // do NOT use "||"
{
return false;
@ -1825,6 +1836,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
return true;
}
otherItr.word--;
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext(1)) // do NOT use "||"
{
return false;
@ -1834,6 +1846,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
if (res >= minElements) {
return true;
}
//noinspection NonShortCircuitBooleanExpression
if (!thisItr.prepareNext() | !otherItr.prepareNext()) // do NOT use "||"
{
return false;
@ -2011,6 +2024,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
/**
* {@inheritDoc}
*/
@SuppressWarnings("CompareToUsesNonFinalVariable")
@Override
public int compareTo(IntSet o)
{

View File

@ -276,7 +276,7 @@ public class RocketMQFirehoseFactory implements FirehoseFactory<ByteBufferInputR
OffsetStore offsetStore = defaultMQPullConsumer.getOffsetStore();
Set<MessageQueue> updated = new HashSet<>();
// calculate offsets according to consuming windows.
for (ConcurrentHashMap.Entry<MessageQueue, ConcurrentSkipListSet<Long>> entry : windows.entrySet()) {
for (Map.Entry<MessageQueue, ConcurrentSkipListSet<Long>> entry : windows.entrySet()) {
while (!entry.getValue().isEmpty()) {
long offset = offsetStore.readOffset(entry.getKey(), ReadOffsetType.MEMORY_FIRST_THEN_STORE);
@ -539,7 +539,7 @@ public class RocketMQFirehoseFactory implements FirehoseFactory<ByteBufferInputR
topicQueueMap.put(topic, mqDivided);
// Remove message queues that are re-assigned to other clients.
Iterator<ConcurrentHashMap.Entry<MessageQueue, ConcurrentSkipListSet<MessageExt>>> it =
Iterator<Map.Entry<MessageQueue, ConcurrentSkipListSet<MessageExt>>> it =
messageQueueTreeSetMap.entrySet().iterator();
while (it.hasNext()) {
if (!mqDivided.contains(it.next().getKey())) {

View File

@ -34,7 +34,6 @@ import org.apache.commons.cli.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Comparator;
import java.util.Date;
import java.util.List;
import java.util.Random;
@ -115,13 +114,11 @@ public class RabbitMQProducerMain
// An extremely silly hack to maintain the above order in the help formatting.
HelpFormatter formatter = new HelpFormatter();
// Add a comparator to the HelpFormatter using the ArrayList above to sort by insertion order.
formatter.setOptionComparator(new Comparator(){
@Override
public int compare(Object o1, Object o2)
{
// I know this isn't fast, but who cares! The list is short.
return optionList.indexOf(o1) - optionList.indexOf(o2);
}
//noinspection ComparatorCombinators -- don't replace with comparingInt() to preserve comments
formatter.setOptionComparator((o1, o2) -> {
// I know this isn't fast, but who cares! The list is short.
//noinspection SuspiciousMethodCalls
return Integer.compare(optionList.indexOf(o1), optionList.indexOf(o2));
});
// Now we can add all the options to an Options instance. This is dumb!

View File

@ -1523,7 +1523,7 @@ public class ApproximateHistogram
public float[] getQuantiles(float[] probabilities)
{
for (float p : probabilities) {
Preconditions.checkArgument(0 < p & p < 1, "quantile probabilities must be strictly between 0 and 1");
Preconditions.checkArgument(0 < p && p < 1, "quantile probabilities must be strictly between 0 and 1");
}
float[] quantiles = new float[probabilities.length];

View File

@ -59,7 +59,7 @@ public class QuantilePostAggregator extends ApproximateHistogramPostAggregator
this.probability = probability;
this.fieldName = fieldName;
if (probability < 0 | probability > 1) {
if (probability < 0 || probability > 1) {
throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", probability);
}
}

View File

@ -50,7 +50,7 @@ public class QuantilesPostAggregator extends ApproximateHistogramPostAggregator
this.probabilities = probabilities;
for (float p : probabilities) {
if (p < 0 | p > 1) {
if (p < 0 || p > 1) {
throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", p);
}
}

View File

@ -223,6 +223,8 @@ public class ApproximateHistogramTest
//for(int i = 0; i < 200; ++i) h.offer((float)(rand.nextGaussian() * 50.0));
long tFold = 0;
int count = 5000000;
// May be a bug that randNums are not used, should be resolved if testFoldSpeed() becomes a jUnit test again
@SuppressWarnings("MismatchedReadAndWriteOfArray")
Float[] randNums = new Float[numRand];
for (int i = 0; i < numRand; i++) {
randNums[i] = (float) rand.nextGaussian();

View File

@ -371,6 +371,7 @@ public class CacheSchedulerTest
lifecycle.stop();
}
while (!cacheManager.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS)) {
// keep waiting
}
checkNoMoreRunning();
@ -395,6 +396,7 @@ public class CacheSchedulerTest
lifecycle.stop();
}
while (!cacheManager.waitForServiceToEnd(1_000, TimeUnit.MILLISECONDS)) {
// keep waiting
}
Assert.assertTrue(scheduler.updatesStarted() >= numWaits);
checkNoMoreRunning();

View File

@ -146,4 +146,13 @@ public class LoadingLookup extends LookupExtractor
return dataFetcher.reverseFetchKeys(value);
}
}
@Override
public String toString()
{
return "LoadingLookup{" +
"dataFetcher=" + dataFetcher +
", id='" + id + '\'' +
'}';
}
}

View File

@ -239,4 +239,12 @@ public class PollingLookup extends LookupExtractor
}
}
@Override
public String toString()
{
return "PollingLookup{" +
"dataFetcher=" + dataFetcher +
", id='" + id + '\'' +
'}';
}
}

View File

@ -193,6 +193,16 @@ public class JdbcDataFetcher implements DataFetcher<String, String>
}
@Override
public String toString()
{
return "JdbcDataFetcher{" +
"table='" + table + '\'' +
", keyColumn='" + keyColumn + '\'' +
", valueColumn='" + valueColumn + '\'' +
'}';
}
private DBI getDbi()
{
return dbi;

View File

@ -72,8 +72,10 @@ public class BucketTest
@Test public void testEquals()
{
//noinspection ObjectEqualsNull
Assert.assertFalse("Object should not be equals to NULL", bucket.equals(null));
Assert.assertFalse("Objects do not have the same Class",bucket.equals(new Integer(0)));
//noinspection EqualsBetweenInconvertibleTypes
Assert.assertFalse("Objects do not have the same Class", bucket.equals(0));
Assert.assertFalse("Objects do not have the same partitionNum",
bucket.equals(new Bucket(shardNum, time, partitionNum + 1)));
Assert.assertFalse("Objects do not have the same shardNum",

View File

@ -26,8 +26,8 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import io.druid.indexing.common.TaskToolbox;
import io.druid.java.util.common.guava.Comparators;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
@ -77,7 +77,7 @@ public class AppendTask extends MergeTaskBase
throws Exception
{
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
Ordering.<String>natural().nullsFirst()
Comparators.naturalNullsFirst()
);
for (DataSegment segment : segments.keySet()) {

View File

@ -27,7 +27,6 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.inject.Injector;
import com.metamx.emitter.EmittingLogger;
@ -38,6 +37,7 @@ import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.task.NoopTask;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.query.filter.DimFilter;
import io.druid.segment.IndexIO;
@ -144,7 +144,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
.submit(new SegmentListUsedAction(dataSource, interval, null));
final Map<DataSegment, File> segmentFileMap = taskToolbox.fetchSegments(usedSegments);
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(
Ordering.<String>natural().nullsFirst()
Comparators.naturalNullsFirst()
);
for (DataSegment segment : usedSegments) {

View File

@ -151,7 +151,7 @@ public class TestNG
private static TestNG m_instance;
private static JCommander m_jCommander;
protected static JCommander m_jCommander;
private List<String> m_commandLineMethods;
protected List<XmlSuite> m_suites = Lists.newArrayList();

View File

@ -86,7 +86,7 @@ public class RemoteTestNG extends TestNG
{
CommandLineArgs cla = new CommandLineArgs();
RemoteArgs ra = new RemoteArgs();
new JCommander(Arrays.asList(cla, ra), args);
m_jCommander = new JCommander(Arrays.asList(cla, ra), args);
m_dontExit = ra.dontExit;
if (cla.port != null && ra.serPort != null) {
throw new TestNGException(

View File

@ -19,16 +19,46 @@
package io.druid.java.util.common.guava;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
import org.joda.time.DateTimeComparator;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Comparator;
/**
*/
public class Comparators
{
private static final Ordering<Object> ALWAYS_EQUAL = new Ordering<Object>()
{
@SuppressWarnings("ComparatorMethodParameterNotUsed")
@Override
public int compare(@Nullable Object left, @Nullable Object right)
{
return 0;
}
};
//CHECKSTYLE.OFF: Regexp
// Ordering.natural().nullsFirst() is generally prohibited, but we need a single exception.
private static final Ordering NATURAL_NULLS_FIRST = Ordering.natural().nullsFirst();
//CHECKSTYLE.ON: Regexp
@SuppressWarnings("unchecked")
public static <T> Ordering<T> alwaysEqual()
{
return (Ordering<T>) ALWAYS_EQUAL;
}
@SuppressWarnings("unchecked")
public static <T extends Comparable<? super T>> Ordering<T> naturalNullsFirst()
{
return NATURAL_NULLS_FIRST;
}
/**
* This is a "reverse" comparator. Positive becomes negative, negative becomes positive and 0 (equal) stays the same.
* This was poorly named as "inverse" as it's not really inverting a true/false relationship

View File

@ -21,10 +21,10 @@ package io.druid.query.aggregation.hyperloglog;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Ordering;
import io.druid.hll.HyperLogLogCollector;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Comparators;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.AggregatorFactoryNotMergeableException;
@ -122,7 +122,7 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
@Override
public Comparator getComparator()
{
return Ordering.<HyperLogLogCollector>natural().nullsFirst();
return Comparators.naturalNullsFirst();
}
@Override

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import io.druid.java.util.common.guava.Comparators;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.cache.CacheKeyBuilder;
@ -58,14 +59,7 @@ public class ConstantPostAggregator implements PostAggregator
@Override
public Comparator getComparator()
{
return new Comparator()
{
@Override
public int compare(Object o1, Object o2)
{
return 0;
}
};
return Comparators.alwaysEqual();
}
@Override

View File

@ -22,8 +22,8 @@ package io.druid.query.aggregation.post;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import io.druid.java.util.common.guava.Comparators;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.cache.CacheKeyBuilder;
@ -83,7 +83,7 @@ public class FinalizingFieldAccessPostAggregator implements PostAggregator
if (aggregators != null && aggregators.containsKey(fieldName)) {
return aggregators.get(fieldName).getComparator();
} else {
return Ordering.natural().nullsFirst();
return Comparators.naturalNullsFirst();
}
}

View File

@ -36,6 +36,7 @@ import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.BaseQuery;
@ -81,8 +82,6 @@ public class GroupByQuery extends BaseQuery<Row>
{
public final static String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst";
private final static Comparator NATURAL_NULLS_FIRST = Ordering.natural().nullsFirst();
private final static Comparator<Row> NON_GRANULAR_TIME_COMP = (Row lhs, Row rhs) -> Longs.compare(
lhs.getTimestampFromEpoch(),
rhs.getTimestampFromEpoch()
@ -327,7 +326,7 @@ public class GroupByQuery extends BaseQuery<Row>
return rowOrdering.compare((Row) lhs, (Row) rhs);
} else {
// Probably bySegment queries
return NATURAL_NULLS_FIRST.compare(lhs, rhs);
return ((Ordering) Comparators.naturalNullsFirst()).compare(lhs, rhs);
}
}
);
@ -563,7 +562,7 @@ public class GroupByQuery extends BaseQuery<Row>
((Number) rhs.getRaw(dimension.getOutputName())).doubleValue()
);
} else {
dimCompare = NATURAL_NULLS_FIRST.compare(
dimCompare = ((Ordering) Comparators.naturalNullsFirst()).compare(
lhs.getRaw(dimension.getOutputName()),
rhs.getRaw(dimension.getOutputName())
);
@ -603,7 +602,7 @@ public class GroupByQuery extends BaseQuery<Row>
if (isNumericField.get(i)) {
if (comparator == StringComparators.NUMERIC) {
dimCompare = NATURAL_NULLS_FIRST.compare(
dimCompare = ((Ordering) Comparators.naturalNullsFirst()).compare(
rhs.getRaw(fieldName),
lhs.getRaw(fieldName)
);

View File

@ -184,4 +184,20 @@ public class GroupByQueryConfig
newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit());
return newConfig;
}
@Override
public String toString()
{
return "GroupByQueryConfig{" +
"defaultStrategy='" + defaultStrategy + '\'' +
", singleThreaded=" + singleThreaded +
", maxIntermediateRows=" + maxIntermediateRows +
", maxResults=" + maxResults +
", bufferGrouperMaxSize=" + bufferGrouperMaxSize +
", bufferGrouperMaxLoadFactor=" + bufferGrouperMaxLoadFactor +
", bufferGrouperInitialBuckets=" + bufferGrouperInitialBuckets +
", maxMergingDictionarySize=" + maxMergingDictionarySize +
", maxOnDiskStorage=" + maxOnDiskStorage +
'}';
}
}

View File

@ -162,4 +162,16 @@ public class LookupExtractionFn extends FunctionalExtraction
result = 31 * result + (isInjective() ? 1 : 0);
return result;
}
@Override
public String toString()
{
return "LookupExtractionFn{" +
"lookup=" + lookup +
", optimize=" + optimize +
", retainMissingValue=" + retainMissingValue +
", replaceMissingValueWith='" + replaceMissingValueWith + '\'' +
", injective=" + injective +
'}';
}
}

View File

@ -22,7 +22,7 @@ package io.druid.query.lookup;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.Ordering;
import io.druid.java.util.common.guava.Comparators;
import java.util.Objects;
@ -30,8 +30,6 @@ import java.util.Objects;
*/
public class LookupExtractorFactoryContainer
{
private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst();
private final String version;
private final LookupExtractorFactory lookupExtractorFactory;
@ -62,7 +60,7 @@ public class LookupExtractorFactoryContainer
return this.lookupExtractorFactory.replaces(other.getLookupExtractorFactory());
}
return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0;
return Comparators.<String>naturalNullsFirst().compare(version, other.getVersion()) > 0;
}
@Override

View File

@ -35,6 +35,7 @@ import io.druid.common.guava.CombiningSequence;
import io.druid.common.utils.JodaUtils;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.MappedSequence;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.nary.BinaryFn;
@ -126,16 +127,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
{
if (query.isMerge()) {
// Merge everything always
return new Ordering<SegmentAnalysis>()
{
@Override
public int compare(
@Nullable SegmentAnalysis left, @Nullable SegmentAnalysis right
)
{
return 0;
}
};
return Comparators.alwaysEqual();
}
return query.getResultOrdering(); // No two elements should be equal, so it should never merge

View File

@ -57,12 +57,12 @@ public class StringComparators
}
}).nullsFirst();
@SuppressWarnings("StringEquality")
@Override
public int compare(String s, String s2)
{
// Avoid conversion to bytes for equal references
// Assuming we mostly compare different strings, checking s.equals(s2) will only make the comparison slower.
//noinspection StringEquality
if (s == s2) {
return 0;
}
@ -319,10 +319,12 @@ public class StringComparators
return Ints.compare(s.length(), s2.length());
}
}).nullsFirst().compound(Ordering.natural());
@Override
public int compare(String s, String s2)
{
// Optimization
//noinspection StringEquality
if (s == s2) {
return 0;
}
@ -372,12 +374,12 @@ public class StringComparators
public static class NumericComparator extends StringComparator
{
@SuppressWarnings("StringEquality")
@Override
public int compare(String o1, String o2)
{
// return if o1 and o2 are the same object
// Assuming we mostly compare different strings, checking o1.equals(o2) will only make the comparison slower.
//noinspection StringEquality
if (o1 == o2) {
return 0;
}

View File

@ -46,6 +46,7 @@ import io.druid.common.utils.SerializerUtils;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.guava.MergeIterable;
import io.druid.java.util.common.guava.nary.BinaryFn;
@ -410,11 +411,8 @@ public class IndexMerger
{
if (rollup) {
return CombiningIterable.create(
new MergeIterable<Rowboat>(
Ordering.<Rowboat>natural().nullsFirst(),
boats
),
Ordering.<Rowboat>natural().nullsFirst(),
new MergeIterable<>(Comparators.naturalNullsFirst(), boats),
Comparators.naturalNullsFirst(),
new RowboatMergeFunction(sortedMetricAggs)
);
} else {
@ -530,10 +528,7 @@ public class IndexMerger
@Nullable final ArrayList<Iterable<Rowboat>> boats
)
{
return new MergeIterable<Rowboat>(
Ordering.<Rowboat>natural().nullsFirst(),
boats
);
return new MergeIterable<>(Comparators.naturalNullsFirst(), boats);
}
};
@ -590,8 +585,8 @@ public class IndexMerger
);
}
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final Map<String, ValueType> valueTypes = Maps.newTreeMap(Comparators.<String>naturalNullsFirst());
final Map<String, String> metricTypeNames = Maps.newTreeMap(Comparators.<String>naturalNullsFirst());
final Map<String, ColumnCapabilitiesImpl> columnCapabilities = Maps.newHashMap();
final List<ColumnCapabilitiesImpl> dimCapabilities = new ArrayList<>();
@ -1019,9 +1014,9 @@ public class IndexMerger
return true;
}
public static <T extends Comparable> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
public static <T extends Comparable<? super T>> ArrayList<T> mergeIndexed(final List<Iterable<T>> indexedLists)
{
Set<T> retVal = Sets.newTreeSet(Ordering.<T>natural().nullsFirst());
Set<T> retVal = Sets.newTreeSet(Comparators.<T>naturalNullsFirst());
for (Iterable<T> indexedList : indexedLists) {
for (T val : indexedList) {
@ -1349,10 +1344,12 @@ public class IndexMerger
{
IntBuffer readOnly = conversions[index].asReadOnlyBuffer();
readOnly.rewind();
for (int i = 0; readOnly.hasRemaining(); i++) {
int i = 0;
while (readOnly.hasRemaining()) {
if (i != readOnly.get()) {
return true;
}
i++;
}
return false;
}

View File

@ -23,16 +23,16 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.io.ByteStreams;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import com.google.inject.Inject;
import io.druid.common.utils.JodaUtils;
import io.druid.java.util.common.io.Closer;
import io.druid.io.ZeroCopyByteArrayOutputStream;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.java.util.common.io.smoosh.SmooshedWriter;
import io.druid.java.util.common.logger.Logger;
@ -157,8 +157,8 @@ public class IndexMergerV9 extends IndexMerger
log.info("Completed factory.json in %,d millis", System.currentTimeMillis() - startTime);
progress.progress();
final Map<String, ValueType> metricsValueTypes = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final Map<String, String> metricTypeNames = Maps.newTreeMap(Ordering.<String>natural().nullsFirst());
final Map<String, ValueType> metricsValueTypes = Maps.newTreeMap(Comparators.<String>naturalNullsFirst());
final Map<String, String> metricTypeNames = Maps.newTreeMap(Comparators.<String>naturalNullsFirst());
final List<ColumnCapabilitiesImpl> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);

View File

@ -30,6 +30,7 @@ import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import java.util.Arrays;
import java.util.Objects;
import java.util.Set;
/**
@ -143,30 +144,27 @@ public class IndexSpec
if (o == null || getClass() != o.getClass()) {
return false;
}
IndexSpec indexSpec = (IndexSpec) o;
if (bitmapSerdeFactory != null
? !bitmapSerdeFactory.equals(indexSpec.bitmapSerdeFactory)
: indexSpec.bitmapSerdeFactory != null) {
return false;
}
if (dimensionCompression != null
? !dimensionCompression.equals(indexSpec.dimensionCompression)
: indexSpec.dimensionCompression != null) {
return false;
}
return !(metricCompression != null
? !metricCompression.equals(indexSpec.metricCompression)
: indexSpec.metricCompression != null);
return Objects.equals(bitmapSerdeFactory, indexSpec.bitmapSerdeFactory) &&
dimensionCompression == indexSpec.dimensionCompression &&
metricCompression == indexSpec.metricCompression &&
longEncoding == indexSpec.longEncoding;
}
@Override
public int hashCode()
{
int result = bitmapSerdeFactory != null ? bitmapSerdeFactory.hashCode() : 0;
result = 31 * result + (dimensionCompression != null ? dimensionCompression.hashCode() : 0);
result = 31 * result + (metricCompression != null ? metricCompression.hashCode() : 0);
return result;
return Objects.hash(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding);
}
@Override
public String toString()
{
return "IndexSpec{" +
"bitmapSerdeFactory=" + bitmapSerdeFactory +
", dimensionCompression=" + dimensionCompression +
", metricCompression=" + metricCompression +
", longEncoding=" + longEncoding +
'}';
}
}

View File

@ -81,7 +81,7 @@ public class StringDimensionHandler implements DimensionHandler<Integer, int[],
) throws SegmentValidationException
{
if (lhs == null || rhs == null) {
if (lhs != rhs) {
if (lhs != null || rhs != null) {
throw new SegmentValidationException(
"Expected nulls, found %s and %s",
Arrays.toString(lhs),

View File

@ -24,12 +24,12 @@ import com.google.common.base.Predicates;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import io.druid.collections.bitmap.BitmapFactory;
import io.druid.collections.bitmap.MutableBitmap;
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Comparators;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.ValueMatcher;
@ -51,7 +51,6 @@ import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -61,8 +60,6 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
{
private static final Function<Object, String> STRING_TRANSFORMER = o -> o != null ? o.toString() : null;
private static final Comparator<String> UNENCODED_COMPARATOR = Ordering.natural().nullsFirst();
private static class DimensionDictionary
{
private String minValue = null;
@ -227,7 +224,7 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
}
if (multiValueHandling.needSorting()) {
// Sort multival row by their unencoded values first.
Arrays.sort(dimensionValues, UNENCODED_COMPARATOR);
Arrays.sort(dimensionValues, Comparators.naturalNullsFirst());
}
final int[] retVal = new int[dimensionValues.length];

View File

@ -32,7 +32,7 @@ import java.io.IOException;
*/
public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<ImmutableBitmap>
{
private static Ordering<ImmutableBitmap> comparator = new Ordering<ImmutableBitmap>()
private static final Ordering<ImmutableBitmap> COMPARATOR = new Ordering<ImmutableBitmap>()
{
@Override
public int compare(
@ -62,7 +62,7 @@ public class BitmapCompressedIndexedInts implements IndexedInts, Comparable<Immu
@Override
public int compareTo(@Nullable ImmutableBitmap otherBitmap)
{
return comparator.compare(immutableBitmap, otherBitmap);
return COMPARATOR.compare(immutableBitmap, otherBitmap);
}
@Override

View File

@ -19,14 +19,13 @@
package io.druid.segment.data;
import com.google.common.collect.Ordering;
import io.druid.java.util.common.guava.Comparators;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class CompressedByteBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<ByteBuffer>
{
public static final Ordering<Comparable> ORDERING = Ordering.natural().nullsFirst();
public static CompressedByteBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{
@ -51,7 +50,7 @@ public class CompressedByteBufferObjectStrategy extends FixedSizeCompressedObjec
@Override
public int compare(ByteBuffer lhs, ByteBuffer rhs)
{
return ORDERING.compare(lhs, rhs);
return Comparators.<ByteBuffer>naturalNullsFirst().compare(lhs, rhs);
}
@Override

View File

@ -19,8 +19,8 @@
package io.druid.segment.data;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Floats;
import io.druid.java.util.common.guava.Comparators;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -50,7 +50,7 @@ public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObje
@Override
public int compare(FloatBuffer lhs, FloatBuffer rhs)
{
return Ordering.natural().nullsFirst().compare(lhs, rhs);
return Comparators.<FloatBuffer>naturalNullsFirst().compare(lhs, rhs);
}
@Override

View File

@ -19,8 +19,8 @@
package io.druid.segment.data;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import io.druid.java.util.common.guava.Comparators;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -28,7 +28,6 @@ import java.nio.IntBuffer;
public class CompressedIntBufferObjectStrategy extends FixedSizeCompressedObjectStrategy<IntBuffer>
{
public static final Ordering<Comparable> ORDERING = Ordering.natural().nullsFirst();
public static CompressedIntBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer)
{
@ -50,7 +49,7 @@ public class CompressedIntBufferObjectStrategy extends FixedSizeCompressedObject
@Override
public int compare(IntBuffer lhs, IntBuffer rhs)
{
return ORDERING.compare(lhs, rhs);
return Comparators.<IntBuffer>naturalNullsFirst().compare(lhs, rhs);
}
@Override

View File

@ -19,8 +19,8 @@
package io.druid.segment.data;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
import io.druid.java.util.common.guava.Comparators;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -50,7 +50,7 @@ public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjec
@Override
public int compare(LongBuffer lhs, LongBuffer rhs)
{
return Ordering.natural().nullsFirst().compare(lhs, rhs);
return Comparators.<LongBuffer>naturalNullsFirst().compare(lhs, rhs);
}
@Override

View File

@ -19,13 +19,13 @@
package io.druid.segment.data;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints;
import io.druid.common.utils.SerializerUtils;
import io.druid.io.ZeroCopyByteArrayOutputStream;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import it.unimi.dsi.fastutil.bytes.ByteArrays;
@ -73,7 +73,6 @@ public class GenericIndexed<T> implements Indexed<T>
static final byte VERSION_TWO = 0x2;
static final byte REVERSE_LOOKUP_ALLOWED = 0x1;
static final byte REVERSE_LOOKUP_DISALLOWED = 0x0;
private final static Ordering<String> NATURAL_STRING_ORDERING = Ordering.natural().nullsFirst();
private static final SerializerUtils SERIALIZER_UTILS = new SerializerUtils();
public static final ObjectStrategy<String> STRING_STRATEGY = new CacheableObjectStrategy<String>()
@ -102,7 +101,7 @@ public class GenericIndexed<T> implements Indexed<T>
@Override
public int compare(String o1, String o2)
{
return NATURAL_STRING_ORDERING.compare(o1, o2);
return Comparators.<String>naturalNullsFirst().compare(o1, o2);
}
};

View File

@ -19,6 +19,8 @@
package io.druid.segment.data;
import io.druid.java.util.common.guava.Comparators;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -53,7 +55,7 @@ public class VSizeCompressedObjectStrategy extends CompressedObjectStrategy<Byte
@Override
public int compare(ByteBuffer lhs, ByteBuffer rhs)
{
return CompressedByteBufferObjectStrategy.ORDERING.compare(lhs, rhs);
return Comparators.<ByteBuffer>naturalNullsFirst().compare(lhs, rhs);
}
@Override

View File

@ -113,7 +113,7 @@ public class ColumnComparisonFilter implements Filter
public static boolean overlap(String[] a, String[] b) {
if (a == null || b == null) {
// They only have overlap if both are null.
return a == b;
return a == null && b == null;
}
if (a.length == 0 && b.length == 0) {
return true;

View File

@ -506,9 +506,10 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
)
{
switch (version) {
case UNCOMPRESSED_MULTI_VALUE:
case UNCOMPRESSED_MULTI_VALUE: {
return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier();
case COMPRESSED:
}
case COMPRESSED: {
if (Feature.MULTI_VALUE.isSet(flags)) {
return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder, fileMapper);
} else if (Feature.MULTI_VALUE_V3.isSet(flags)) {
@ -516,6 +517,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
} else {
throw new IAE("Unrecognized multi-value flag[%d]", flags);
}
}
default:
throw new IAE("Unsupported multi-value version[%s]", version);
}

View File

@ -286,11 +286,9 @@ public class CacheKeyBuilderTest
private static void assertNotEqualsEachOther(List<byte[]> keys)
{
for (byte[] k1 : keys) {
for (byte[] k2 : keys) {
if (k1 != k2) {
assertFalse(Arrays.equals(k1, k2));
}
for (int i = 0; i < keys.size(); i++) {
for (int j = i + 1; j < keys.size(); j++) {
assertFalse(Arrays.equals(keys.get(i), keys.get(j)));
}
}
}

View File

@ -21,6 +21,8 @@ package io.druid.query.groupby.epinephelinae;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import org.junit.Assert;
import org.junit.Test;
@ -195,9 +197,9 @@ public class ByteBufferMinMaxOffsetHeapTest
{
int limit = 100;
ArrayList<Integer> values = Lists.newArrayList(
IntList values = new IntArrayList(new int[] {
1, 20, 1000, 2, 3, 30, 40, 10, 11, 12, 13, 300, 400, 500, 600
);
});
ByteBuffer myBuffer = ByteBuffer.allocate(1000000);
ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.<Integer>natural(), null);
@ -212,7 +214,7 @@ public class ByteBufferMinMaxOffsetHeapTest
Assert.assertTrue(heap.isIntact());
Collections.sort(values);
values.remove((Number) 12);
values.rem(12);
List<Integer> actual = Lists.newArrayList();
for (int i = 0; i < values.size(); i++){
@ -228,10 +230,10 @@ public class ByteBufferMinMaxOffsetHeapTest
{
int limit = 100;
ArrayList<Integer> values = Lists.newArrayList(
IntList values = new IntArrayList(new int[] {
1, 20, 1000, 2, 3, 30, 40, 10, 11, 12, 13, 300, 400, 500, 600, 4, 5,
6, 7, 8, 9, 4, 5, 200, 250
);
});
ByteBuffer myBuffer = ByteBuffer.allocate(1000000);
ByteBufferMinMaxOffsetHeap heap = new ByteBufferMinMaxOffsetHeap(myBuffer, limit, Ordering.<Integer>natural(), null);
@ -245,7 +247,7 @@ public class ByteBufferMinMaxOffsetHeapTest
Assert.assertTrue(heap.isIntact());
Collections.sort(values);
values.remove((Number) 2);
values.rem(2);
Assert.assertTrue(heap.isIntact());
List<Integer> actual = Lists.newArrayList();

View File

@ -26,12 +26,12 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import io.druid.data.input.MapBasedInputRow;
import io.druid.hll.HyperLogLogHash;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
@ -444,7 +444,7 @@ public class SchemalessIndexTest
List<File> filesToMap = makeFilesToMap(tmpFile, files);
VersionedIntervalTimeline<Integer, File> timeline = new VersionedIntervalTimeline<Integer, File>(
Ordering.natural().nullsFirst()
Comparators.naturalNullsFirst()
);
ShardSpec noneShardSpec = NoneShardSpec.instance();

View File

@ -645,4 +645,13 @@ public class DirectDruidClient<T> implements QueryRunner<T>
}
}
}
@Override
public String toString()
{
return "DirectDruidClient{" +
"host='" + host + '\'' +
", isSmile=" + isSmile +
'}';
}
}

View File

@ -172,6 +172,16 @@ public class ArbitraryGranularitySpec implements GranularitySpec
return result;
}
@Override
public String toString()
{
return "ArbitraryGranularitySpec{" +
"intervals=" + intervals +
", queryGranularity=" + queryGranularity +
", rollup=" + rollup +
'}';
}
@Override
public GranularitySpec withIntervals(List<Interval> inputIntervals) {
return new ArbitraryGranularitySpec(queryGranularity, rollup, inputIntervals);

View File

@ -120,7 +120,7 @@ public class FireHydrant
{
return "FireHydrant{" +
"index=" + index +
", queryable=" + adapter +
", queryable=" + adapter.getIdentifier() +
", count=" + count +
'}';
}

View File

@ -145,6 +145,7 @@ public class CostBalancerStrategy implements BalancerStrategy
beta = x1 - y0;
gamma = y1 - y0;
}
//noinspection SuspiciousNameCombination
return intervalCost(y0, y0, y1) + // cost(A, Y)
intervalCost(beta, beta, gamma) + // cost(B, C)
2 * (beta + FastMath.exp(-beta) - 1); // cost(B, B)

View File

@ -22,7 +22,7 @@ package io.druid.server.lookup.cache;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.Ordering;
import io.druid.java.util.common.guava.Comparators;
import java.util.Map;
import java.util.Objects;
@ -34,8 +34,6 @@ import java.util.Objects;
*/
public class LookupExtractorFactoryMapContainer
{
private final static Ordering VERSION_COMPARATOR = Ordering.natural().nullsFirst();
private final String version;
private final Map<String, Object> lookupExtractorFactory;
@ -66,7 +64,7 @@ public class LookupExtractorFactoryMapContainer
return false;
}
return VERSION_COMPARATOR.compare(version, other.getVersion()) > 0;
return Comparators.<String>naturalNullsFirst().compare(version, other.getVersion()) > 0;
}
@Override

View File

@ -60,6 +60,7 @@ import io.druid.java.util.common.Pair;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.granularity.PeriodGranularity;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.guava.MergeIterable;
import io.druid.java.util.common.guava.Sequence;
@ -2167,7 +2168,7 @@ public class CachingClusteredClientTest
for (int i = 0; i < numTimesToQuery; ++i) {
TestHelper.assertExpectedResults(
new MergeIterable<>(
Ordering.<Result<Object>>natural().nullsFirst(),
Comparators.naturalNullsFirst(),
FunctionalIterable
.create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd))
.transformCat(