Avoid using the default system Locale and printing to System.out in production code (#4409)

* Avoid usages of Default system Locale and printing to System.out or System.err in production code

* Fix Charset in DruidKerberosUtil

* Remove redundant string format in GenericIndexed

* Rename StringUtils.safeFormat() to unimportantSafeFormat(); add StringUtils.format() which fails as well as String.format()

* Fix testSafeFormat()

* More fixes of redundant StringUtils.format() inside ISE

* Rename unimportantSafeFormat() to nonStrictFormat()
This commit is contained in:
Roman Leventov 2017-06-29 16:06:19 -05:00 committed by Gian Merlino
parent a5651ea474
commit 9ae457f7ad
339 changed files with 1323 additions and 1561 deletions

View File

@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.StringUtils;
/**
*/
@ -57,13 +58,13 @@ public abstract class DimensionSchema
@Override
public String toString()
{
return this.name().toUpperCase();
return StringUtils.toUpperCase(this.name());
}
@JsonCreator
public static ValueType fromString(String name)
{
return valueOf(name.toUpperCase());
return valueOf(StringUtils.toUpperCase(name));
}
}
@ -85,13 +86,13 @@ public abstract class DimensionSchema
@JsonValue
public String toString()
{
return name().toUpperCase();
return StringUtils.toUpperCase(name());
}
@JsonCreator
public static MultiValueHandling fromString(String name)
{
return name == null ? ofDefault() : valueOf(name.toUpperCase());
return name == null ? ofDefault() : valueOf(StringUtils.toUpperCase(name));
}
// this can be system configuration

View File

@ -21,6 +21,7 @@ package io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonValue;
import io.druid.java.util.common.StringUtils;
public enum JSONPathFieldType
{
@ -31,12 +32,12 @@ public enum JSONPathFieldType
@Override
public String toString()
{
return this.name().toLowerCase();
return StringUtils.toLowerCase(this.name());
}
@JsonCreator
public static JSONPathFieldType fromString(String name)
{
return valueOf(name.toUpperCase());
return valueOf(StringUtils.toUpperCase(name));
}
}

View File

@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.parsers.ParseException;
import org.joda.time.DateTime;
@ -62,7 +63,7 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
if (timestamp == null) {
final String input = theMap.toString();
throw new NullPointerException(
String.format(
StringUtils.format(
"Null timestamp in input: %s",
input.length() < 100 ? input : input.substring(0, 100) + "..."
)

View File

@ -34,6 +34,7 @@ import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.google.inject.ProvisionException;
import com.google.inject.spi.Message;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import javax.validation.ConstraintViolation;
@ -102,7 +103,7 @@ public class JsonConfigurator
}
catch (IllegalArgumentException e) {
throw new ProvisionException(
String.format("Problem parsing object at prefix[%s]: %s.", propertyPrefix, e.getMessage()), e
StringUtils.format("Problem parsing object at prefix[%s]: %s.", propertyPrefix, e.getMessage()), e
);
}
@ -122,7 +123,7 @@ public class JsonConfigurator
final Field theField = beanClazz.getDeclaredField(fieldName);
if (theField.getAnnotation(JacksonInject.class) != null) {
path = String.format(" -- Injected field[%s] not bound!?", fieldName);
path = StringUtils.format(" -- Injected field[%s] not bound!?", fieldName);
break;
}
@ -142,7 +143,7 @@ public class JsonConfigurator
throw Throwables.propagate(e);
}
messages.add(String.format("%s - %s", path, violation.getMessage()));
messages.add(StringUtils.format("%s - %s", path, violation.getMessage()));
}
throw new ProvisionException(
@ -153,7 +154,7 @@ public class JsonConfigurator
@Override
public Message apply(String input)
{
return new Message(String.format("%s%s", propertyBase, input));
return new Message(StringUtils.format("%s%s", propertyBase, input));
}
}
)
@ -175,7 +176,7 @@ public class JsonConfigurator
final AnnotatedField field = beanDef.getField();
if (field == null || !field.hasAnnotation(JsonProperty.class)) {
throw new ProvisionException(
String.format(
StringUtils.format(
"JsonConfigurator requires Jackson-annotated Config objects to have field annotations. %s doesn't",
clazz
)

View File

@ -30,6 +30,7 @@ import com.google.inject.TypeLiteral;
import com.google.inject.binder.ScopedBindingBuilder;
import com.google.inject.multibindings.MapBinder;
import com.google.inject.util.Types;
import io.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
import java.lang.reflect.ParameterizedType;
@ -188,7 +189,7 @@ public class PolyBind
if (implName == null) {
if (defaultPropertyValue == null) {
if (defaultKey == null) {
throw new ProvisionException(String.format("Some value must be configured for [%s]", key));
throw new ProvisionException(StringUtils.format("Some value must be configured for [%s]", key));
}
return injector.getInstance(defaultKey);
}
@ -198,7 +199,7 @@ public class PolyBind
if (provider == null) {
throw new ProvisionException(
String.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet())
StringUtils.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet())
);
}

View File

@ -21,6 +21,7 @@ package io.druid.segment;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import io.druid.java.util.common.IOE;
import java.io.File;
import java.io.FileInputStream;
@ -47,11 +48,6 @@ public class SegmentUtils
return version;
}
throw new IOException(
String.format(
"Invalid segment dir [%s]. Can't find either of version.bin or index.drd.",
inDir
)
);
throw new IOE("Invalid segment dir [%s]. Can't find either of version.bin or index.drd.", inDir);
}
}

View File

@ -20,6 +20,7 @@
package io.druid.segment.loading;
import com.google.common.base.Joiner;
import io.druid.java.util.common.StringUtils;
import io.druid.timeline.DataSegment;
import java.io.File;
@ -41,7 +42,7 @@ public interface DataSegmentPusher
return getDefaultStorageDir(dataSegment);
}
default String makeIndexPathName(DataSegment dataSegment, String indexName) {
return String.format("./%s/%s", getStorageDir(dataSegment),indexName);
return StringUtils.format("./%s/%s", getStorageDir(dataSegment), indexName);
}
// Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/
@ -51,11 +52,7 @@ public interface DataSegmentPusher
static String getDefaultStorageDir(DataSegment segment) {
return JOINER.join(
segment.getDataSource(),
String.format(
"%s_%s",
segment.getInterval().getStart(),
segment.getInterval().getEnd()
),
StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()),
segment.getVersion(),
segment.getShardSpec().getPartitionNum()
);

View File

@ -19,6 +19,8 @@
package io.druid.segment.loading;
import io.druid.java.util.common.StringUtils;
/**
*/
public class SegmentLoadingException extends Exception
@ -28,7 +30,7 @@ public class SegmentLoadingException extends Exception
Object... objs
)
{
super(String.format(formatString, objs));
super(StringUtils.nonStrictFormat(formatString, objs));
}
public SegmentLoadingException(
@ -37,6 +39,6 @@ public class SegmentLoadingException extends Exception
Object... objs
)
{
super(String.format(formatString, objs), cause);
super(StringUtils.nonStrictFormat(formatString, objs), cause);
}
}

View File

@ -20,10 +20,9 @@
package io.druid.timeline;
import com.google.common.base.Function;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.format.DateTimeFormatter;
@ -73,7 +72,7 @@ public class DataSegmentUtils
*/
public static SegmentIdentifierParts valueOf(String dataSource, String identifier)
{
if (!identifier.startsWith(String.format("%s_", dataSource))) {
if (!identifier.startsWith(StringUtils.format("%s_", dataSource))) {
return null;
}

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.io.ByteSink;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
@ -48,6 +49,7 @@ import java.util.Map;
public class FloatCompressionBenchmarkFileGenerator
{
private static final Logger log = new Logger(FloatCompressionBenchmarkFileGenerator.class);
public static final int ROW_NUM = 5000000;
public static final List<CompressedObjectStrategy.CompressionStrategy> compressions =
ImmutableList.of(
@ -143,7 +145,7 @@ public class FloatCompressionBenchmarkFileGenerator
for (Map.Entry<String, BenchmarkColumnValueGenerator> entry : generators.entrySet()) {
for (CompressedObjectStrategy.CompressionStrategy compression : compressions) {
String name = entry.getKey() + "-" + compression.toString();
System.out.print(name + ": ");
log.info("%s: ", name);
File compFile = new File(dir, name);
compFile.delete();
File dataFile = new File(dir, entry.getKey());
@ -184,7 +186,7 @@ public class FloatCompressionBenchmarkFileGenerator
iopeon.close();
br.close();
}
System.out.print(compFile.length() / 1024 + "\n");
log.info("%d", compFile.length() / 1024);
}
}
}

View File

@ -22,6 +22,7 @@ package io.druid.benchmark;
import com.google.common.collect.ImmutableMap;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.StringUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
@ -66,14 +67,14 @@ public class IncrementalIndexRowTypeBenchmark
for (int i = 0; i < dimensionCount; ++i) {
ingestAggregatorFactories.add(
new LongSumAggregatorFactory(
String.format("sumResult%s", i),
String.format("Dim_%s", i)
StringUtils.format("sumResult%s", i),
StringUtils.format("Dim_%s", i)
)
);
ingestAggregatorFactories.add(
new DoubleSumAggregatorFactory(
String.format("doubleSumResult%s", i),
String.format("Dim_%s", i)
StringUtils.format("doubleSumResult%s", i),
StringUtils.format("Dim_%s", i)
)
);
}
@ -85,7 +86,7 @@ public class IncrementalIndexRowTypeBenchmark
List<String> dimensionList = new ArrayList<String>(dimensionCount);
ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
for (int i = 0; i < dimensionCount; i++) {
String dimName = String.format("Dim_%d", i);
String dimName = StringUtils.format("Dim_%d", i);
dimensionList.add(dimName);
builder.put(dimName, rng.nextLong());
}
@ -97,7 +98,7 @@ public class IncrementalIndexRowTypeBenchmark
List<String> dimensionList = new ArrayList<String>(dimensionCount);
ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
for (int i = 0; i < dimensionCount; i++) {
String dimName = String.format("Dim_%d", i);
String dimName = StringUtils.format("Dim_%d", i);
dimensionList.add(dimName);
builder.put(dimName, rng.nextFloat());
}
@ -109,7 +110,7 @@ public class IncrementalIndexRowTypeBenchmark
List<String> dimensionList = new ArrayList<String>(dimensionCount);
ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
for (int i = 0; i < dimensionCount; i++) {
String dimName = String.format("Dim_%d", i);
String dimName = StringUtils.format("Dim_%d", i);
dimensionList.add(dimName);
builder.put(dimName, String.valueOf(rng.nextLong()));
}

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.io.ByteSink;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
@ -48,6 +49,7 @@ import java.util.Map;
public class LongCompressionBenchmarkFileGenerator
{
private static final Logger log = new Logger(LongCompressionBenchmarkFileGenerator.class);
public static final int ROW_NUM = 5000000;
public static final List<CompressedObjectStrategy.CompressionStrategy> compressions =
ImmutableList.of(CompressedObjectStrategy.CompressionStrategy.LZ4,
@ -135,7 +137,7 @@ public class LongCompressionBenchmarkFileGenerator
for (CompressedObjectStrategy.CompressionStrategy compression : compressions) {
for (CompressionFactory.LongEncodingStrategy encoding : encodings) {
String name = entry.getKey() + "-" + compression.toString() + "-" + encoding.toString();
System.out.print(name + ": ");
log.info("%s: ", name);
File compFile = new File(dir, name);
compFile.delete();
File dataFile = new File(dir, entry.getKey());
@ -177,7 +179,7 @@ public class LongCompressionBenchmarkFileGenerator
iopeon.close();
br.close();
}
System.out.print(compFile.length() / 1024 + "\n");
log.info("%d", compFile.length() / 1024);
}
}
}

View File

@ -20,6 +20,7 @@
package io.druid.benchmark;
import com.google.common.io.Files;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.data.VSizeLongSerde;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -50,6 +51,7 @@ import java.util.concurrent.TimeUnit;
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public class VSizeSerdeBenchmark
{
private static final Logger log = new Logger(VSizeSerdeBenchmark.class);
@Param({"500000"})
private int values;
@ -102,7 +104,7 @@ public class VSizeSerdeBenchmark
public void tearDown()
{
dummy.delete();
System.out.println(sum);
log.info("%d", sum);
}
@Benchmark

View File

@ -21,6 +21,7 @@ package io.druid.collections.bitmap;
import com.google.common.collect.Iterables;
import io.druid.extendedset.intset.ImmutableConciseSet;
import io.druid.java.util.common.ISE;
import java.nio.ByteBuffer;
import java.util.Collection;
@ -92,7 +93,7 @@ public class ConciseBitmapFactory implements BitmapFactory
public ImmutableBitmap makeImmutableBitmap(MutableBitmap mutableBitmap)
{
if (!(mutableBitmap instanceof WrappedConciseBitmap)) {
throw new IllegalStateException(String.format("Cannot convert [%s]", mutableBitmap.getClass()));
throw new ISE("Cannot convert [%s]", mutableBitmap.getClass());
}
return new WrappedImmutableConciseBitmap(
ImmutableConciseSet.newImmutableFromMutable(

View File

@ -21,6 +21,7 @@ package io.druid.collections.bitmap;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.ISE;
import org.roaringbitmap.RoaringBitmap;
import org.roaringbitmap.buffer.BufferFastAggregation;
import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
@ -125,7 +126,7 @@ public class RoaringBitmapFactory implements BitmapFactory
public ImmutableBitmap makeImmutableBitmap(MutableBitmap mutableBitmap)
{
if (!(mutableBitmap instanceof WrappedRoaringBitmap)) {
throw new IllegalStateException(String.format("Cannot convert [%s]", mutableBitmap.getClass()));
throw new ISE("Cannot convert [%s]", mutableBitmap.getClass());
}
try {
return ((WrappedRoaringBitmap) mutableBitmap).toImmutableBitmap();

View File

@ -19,6 +19,8 @@
package io.druid.collections.bitmap;
import io.druid.java.util.common.IAE;
import java.nio.ByteBuffer;
import java.util.BitSet;
@ -61,12 +63,10 @@ public class WrappedBitSetBitmap extends WrappedImmutableBitSetBitmap implements
WrappedBitSetBitmap bitSet = (WrappedBitSetBitmap) mutableBitmap;
this.bitmap.or(bitSet.bitmap);
} else {
throw new IllegalArgumentException(
String.format(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
)
throw new IAE(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
);
}
}
@ -78,12 +78,10 @@ public class WrappedBitSetBitmap extends WrappedImmutableBitSetBitmap implements
WrappedBitSetBitmap bitSet = (WrappedBitSetBitmap) mutableBitmap;
this.bitmap.and(bitSet.bitmap);
} else {
throw new IllegalArgumentException(
String.format(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
)
throw new IAE(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
);
}
}
@ -95,12 +93,10 @@ public class WrappedBitSetBitmap extends WrappedImmutableBitSetBitmap implements
WrappedBitSetBitmap bitSet = (WrappedBitSetBitmap) mutableBitmap;
this.bitmap.xor(bitSet.bitmap);
} else {
throw new IllegalArgumentException(
String.format(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
)
throw new IAE(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
);
}
}
@ -112,12 +108,10 @@ public class WrappedBitSetBitmap extends WrappedImmutableBitSetBitmap implements
WrappedBitSetBitmap bitSet = (WrappedBitSetBitmap) mutableBitmap;
this.bitmap.andNot(bitSet.bitmap);
} else {
throw new IllegalArgumentException(
String.format(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
)
throw new IAE(
"Unknown class type: %s expected %s",
mutableBitmap.getClass().getCanonicalName(),
WrappedBitSetBitmap.class.getCanonicalName()
);
}
}

View File

@ -205,7 +205,7 @@ public class WrappedRoaringBitmap implements MutableBitmap
);
}
catch (IOException e) {
e.printStackTrace(); // impossible in theory
throw new RuntimeException(e); // impossible in theory
}
}

View File

@ -19,17 +19,12 @@
package io.druid.collections.spatial;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
/**
*/
public class RTreeUtils
{
private static ObjectMapper jsonMapper = new ObjectMapper();
public static double getEnclosingArea(Node a, Node b)
{
@ -76,177 +71,4 @@ public class RTreeUtils
}
}
public static Iterable<ImmutablePoint> getBitmaps(ImmutableRTree tree)
{
return depthFirstSearch(tree.getRoot());
}
public static Iterable<ImmutablePoint> depthFirstSearch(ImmutableNode node)
{
if (node.isLeaf()) {
return Iterables.transform(
node.getChildren(),
new Function<ImmutableNode, ImmutablePoint>()
{
@Override
public ImmutablePoint apply(ImmutableNode tNode)
{
return new ImmutablePoint(tNode);
}
}
);
} else {
return Iterables.concat(
Iterables.transform(
node.getChildren(),
new Function<ImmutableNode, Iterable<ImmutablePoint>>()
{
@Override
public Iterable<ImmutablePoint> apply(ImmutableNode child)
{
return depthFirstSearch(child);
}
}
)
);
}
}
public static void print(RTree tree)
{
System.out.printf("numDims : %d%n", tree.getNumDims());
try {
printRTreeNode(tree.getRoot(), 0);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public static void print(ImmutableRTree tree)
{
System.out.printf("numDims : %d%n", tree.getNumDims());
try {
printNode(tree.getRoot(), 0);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public static void printRTreeNode(Node node, int level) throws Exception
{
System.out.printf(
"%sminCoords: %s, maxCoords: %s, numChildren: %d, isLeaf:%s%n",
makeDashes(level),
jsonMapper.writeValueAsString(node.getMinCoordinates()),
jsonMapper.writeValueAsString(
node.getMaxCoordinates()
),
node.getChildren().size(),
node.isLeaf()
);
if (node.isLeaf()) {
for (Node child : node.getChildren()) {
Point point = (Point) (child);
System.out
.printf(
"%scoords: %s, conciseSet: %s%n",
makeDashes(level),
jsonMapper.writeValueAsString(point.getCoords()),
point.getBitmap()
);
}
} else {
level++;
for (Node child : node.getChildren()) {
printRTreeNode(child, level);
}
}
}
public static boolean verifyEnclose(Node node)
{
for (Node child : node.getChildren()) {
for (int i = 0; i < node.getNumDims(); i++) {
if (child.getMinCoordinates()[i] < node.getMinCoordinates()[i]
|| child.getMaxCoordinates()[i] > node.getMaxCoordinates()[i]) {
return false;
}
}
}
if (!node.isLeaf()) {
for (Node child : node.getChildren()) {
if (!verifyEnclose(child)) {
return false;
}
}
}
return true;
}
public static boolean verifyEnclose(ImmutableNode node)
{
for (ImmutableNode child : node.getChildren()) {
for (int i = 0; i < node.getNumDims(); i++) {
if (child.getMinCoordinates()[i] < node.getMinCoordinates()[i]
|| child.getMaxCoordinates()[i] > node.getMaxCoordinates()[i]) {
return false;
}
}
}
if (!node.isLeaf()) {
for (ImmutableNode child : node.getChildren()) {
if (!verifyEnclose(child)) {
return false;
}
}
}
return true;
}
private static void printNode(ImmutableNode node, int level) throws Exception
{
System.out.printf(
"%sminCoords: %s, maxCoords: %s, numChildren: %d, isLeaf: %s%n",
makeDashes(level),
jsonMapper.writeValueAsString(node.getMinCoordinates()),
jsonMapper.writeValueAsString(
node.getMaxCoordinates()
),
node.getNumChildren(),
node.isLeaf()
);
if (node.isLeaf()) {
for (ImmutableNode immutableNode : node.getChildren()) {
ImmutablePoint point = new ImmutablePoint(immutableNode);
System.out
.printf(
"%scoords: %s, conciseSet: %s%n",
makeDashes(level),
jsonMapper.writeValueAsString(point.getCoords()),
point.getImmutableBitmap()
);
}
} else {
level++;
for (ImmutableNode immutableNode : node.getChildren()) {
printNode(immutableNode, level);
}
}
}
private static String makeDashes(int level)
{
String retVal = "";
for (int i = 0; i < level; i++) {
retVal += "-";
}
return retVal;
}
}

View File

@ -19,12 +19,11 @@
package io.druid.collections.bitmap;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
import com.carrotsearch.junitbenchmarks.BenchmarkRule;
import com.carrotsearch.junitbenchmarks.Clock;
import com.google.common.collect.Lists;
import io.druid.extendedset.intset.ImmutableConciseSet;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -33,12 +32,12 @@ import org.roaringbitmap.buffer.BufferFastAggregation;
import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
import org.roaringbitmap.buffer.MutableRoaringBitmap;
import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
import com.carrotsearch.junitbenchmarks.BenchmarkRule;
import com.carrotsearch.junitbenchmarks.Clock;
import com.google.common.collect.Lists;
import io.druid.extendedset.intset.ImmutableConciseSet;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Locale;
import java.util.Random;
@BenchmarkOptions(clock = Clock.NANO_TIME, benchmarkRounds = 50)
@ -102,11 +101,12 @@ public class BitmapBenchmark
System.out.println("");
System.out.println("## " + name);
System.out.println("");
System.out.printf(" d = %06.5f | Concise | Roaring" + System.lineSeparator(), density);
System.out.printf(Locale.ENGLISH, " d = %06.5f | Concise | Roaring%n", density);
System.out.println("-------------|---------|---------");
System.out.printf("Count | %5d | %5d " + System.lineSeparator(), conciseCount, roaringCount);
System.out.printf(Locale.ENGLISH, "Count | %5d | %5d %n", conciseCount, roaringCount);
System.out.printf(
"Average size | %5d | %5d " + System.lineSeparator(),
Locale.ENGLISH,
"Average size | %5d | %5d %n",
totalConciseBytes / conciseCount,
totalRoaringBytes / roaringCount
);

View File

@ -19,18 +19,17 @@
package io.druid.collections.bitmap;
import java.util.BitSet;
import com.carrotsearch.junitbenchmarks.annotation.BenchmarkHistoryChart;
import com.carrotsearch.junitbenchmarks.annotation.LabelType;
import io.druid.extendedset.intset.ConciseSet;
import io.druid.extendedset.intset.ImmutableConciseSet;
import io.druid.java.util.common.StringUtils;
import io.druid.test.annotation.Benchmark;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
import org.roaringbitmap.buffer.MutableRoaringBitmap;
import com.carrotsearch.junitbenchmarks.annotation.BenchmarkHistoryChart;
import com.carrotsearch.junitbenchmarks.annotation.LabelType;
import io.druid.test.annotation.Benchmark;
import io.druid.extendedset.intset.ConciseSet;
import io.druid.extendedset.intset.ImmutableConciseSet;
import java.util.BitSet;
@Category({Benchmark.class})
@BenchmarkHistoryChart(labelWith = LabelType.CUSTOM_KEY, maxRuns = 20)
@ -43,7 +42,7 @@ public class RangeBitmapBenchmarkTest extends BitmapBenchmark
@BeforeClass
public static void prepareRandomRanges() throws Exception
{
System.setProperty("jub.customkey", String.format("%06.5f", DENSITY));
System.setProperty("jub.customkey", StringUtils.format("%06.5f", DENSITY));
reset();
final BitSet expectedUnion = new BitSet();

View File

@ -19,18 +19,17 @@
package io.druid.collections.bitmap;
import java.util.BitSet;
import com.carrotsearch.junitbenchmarks.annotation.BenchmarkHistoryChart;
import com.carrotsearch.junitbenchmarks.annotation.LabelType;
import io.druid.extendedset.intset.ConciseSet;
import io.druid.extendedset.intset.ImmutableConciseSet;
import io.druid.java.util.common.StringUtils;
import io.druid.test.annotation.Benchmark;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
import org.roaringbitmap.buffer.MutableRoaringBitmap;
import com.carrotsearch.junitbenchmarks.annotation.BenchmarkHistoryChart;
import com.carrotsearch.junitbenchmarks.annotation.LabelType;
import io.druid.test.annotation.Benchmark;
import io.druid.extendedset.intset.ConciseSet;
import io.druid.extendedset.intset.ImmutableConciseSet;
import java.util.BitSet;
@Category({Benchmark.class})
@BenchmarkHistoryChart(labelWith = LabelType.CUSTOM_KEY, maxRuns = 20)
@ -43,7 +42,7 @@ public class UniformBitmapBenchmarkTest extends BitmapBenchmark
@BeforeClass
public static void prepareMostlyUniform() throws Exception
{
System.setProperty("jub.customkey", String.format("%05.4f", DENSITY));
System.setProperty("jub.customkey", StringUtils.format("%05.4f", DENSITY));
reset();
final BitSet expectedUnion = new BitSet();

View File

@ -36,6 +36,7 @@ import org.junit.Test;
import org.roaringbitmap.IntIterator;
import java.nio.ByteBuffer;
import java.util.Locale;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@ -563,13 +564,13 @@ public class ImmutableRTreeTest
tree.insert(new float[]{(float) (rand.nextDouble() * 100), (float) (rand.nextDouble() * 100)}, i);
}
long stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: insert = %,d ms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: insert = %,d ms%n", numPoints, stop);
stopwatch.reset().start();
ImmutableRTree searchTree = ImmutableRTree.newImmutableFromMutable(tree);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: size = %,d bytes%n", numPoints, searchTree.toBytes().length);
System.out.printf("[%,d]: buildImmutable = %,d ms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: size = %,d bytes%n", numPoints, searchTree.toBytes().length);
System.out.printf(Locale.ENGLISH, "[%,d]: buildImmutable = %,d ms%n", numPoints, stop);
stopwatch.reset().start();
@ -578,14 +579,14 @@ public class ImmutableRTreeTest
Iterables.size(points);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: search = %,dms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: search = %,dms%n", numPoints, stop);
stopwatch.reset().start();
ImmutableBitmap finalSet = bf.union(points);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: union of %,d points in %,d ms%n", numPoints, finalSet.size(), stop);
System.out.printf(Locale.ENGLISH, "[%,d]: union of %,d points in %,d ms%n", numPoints, finalSet.size(), stop);
}
catch (Exception e) {
throw Throwables.propagate(e);
@ -613,13 +614,13 @@ public class ImmutableRTreeTest
tree.insert(new float[]{(float) (rand.nextDouble() * 100), (float) (rand.nextDouble() * 100)}, i);
}
long stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: insert = %,d ms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: insert = %,d ms%n", numPoints, stop);
stopwatch.reset().start();
ImmutableRTree searchTree = ImmutableRTree.newImmutableFromMutable(tree);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: size = %,d bytes%n", numPoints, searchTree.toBytes().length);
System.out.printf("[%,d]: buildImmutable = %,d ms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: size = %,d bytes%n", numPoints, searchTree.toBytes().length);
System.out.printf(Locale.ENGLISH, "[%,d]: buildImmutable = %,d ms%n", numPoints, stop);
stopwatch.reset().start();
@ -634,14 +635,14 @@ public class ImmutableRTreeTest
Iterables.size(points);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: search = %,dms%n", numPoints, stop);
System.out.printf(Locale.ENGLISH, "[%,d]: search = %,dms%n", numPoints, stop);
stopwatch.reset().start();
ImmutableBitmap finalSet = bf.union(points);
stop = stopwatch.elapsed(TimeUnit.MILLISECONDS);
System.out.printf("[%,d]: union of %,d points in %,d ms%n", numPoints, finalSet.size(), stop);
System.out.printf(Locale.ENGLISH, "[%,d]: union of %,d points in %,d ms%n", numPoints, finalSet.size(), stop);
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -33,6 +33,10 @@
<suppress checks="AvoidStaticImport" files="[\\/]src[\\/]test[\\/]" />
<suppress checks="Header" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="NeedBraces" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="UnusedImports" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="AvoidStarImport" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<suppress checks="FileTabCharacter" files="[\\/]target[\\/]generated-test-sources[\\/]" />
<!-- extendedset is a fork of Alessandro Colantonio's CONCISE (COmpressed 'N' Composable Integer SEt) repository and licensed to Metamarkets under a CLA is not true. -->
<suppress checks="Header" files="[\\/]extendedset[\\/]" />

View File

@ -20,6 +20,7 @@
package io.druid.common.config;
import com.google.common.base.Throwables;
import io.druid.java.util.common.ISE;
import org.apache.logging.log4j.core.util.Cancellable;
import org.apache.logging.log4j.core.util.ShutdownCallbackRegistry;
@ -90,7 +91,7 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, org.apache.loggi
public void start()
{
if (!state.compareAndSet(State.INITIALIZED, State.STARTED)) { // Skip STARTING
throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.INITIALIZED, state.get()));
throw new ISE("Expected state [%s] found [%s]", State.INITIALIZED, state.get());
}
}
@ -100,7 +101,7 @@ public class Log4jShutdown implements ShutdownCallbackRegistry, org.apache.loggi
if (!state.compareAndSet(State.STARTED, State.STOPPING)) {
State current = state.waitForTransition(State.STOPPING, State.STOPPED, SHUTDOWN_WAIT_TIMEOUT);
if (current != State.STOPPED) {
throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.STARTED, current));
throw new ISE("Expected state [%s] found [%s]", State.STARTED, current);
}
return;
}

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
import com.google.common.io.CharStreams;
import com.google.common.io.InputSupplier;
import com.google.common.primitives.Longs;
import io.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
import java.io.BufferedReader;
@ -49,7 +50,7 @@ public class GuavaUtils
@Override
public String apply(@Nullable String input)
{
return String.format(formatString, input);
return StringUtils.format(formatString, input);
}
};
}

View File

@ -47,6 +47,6 @@ public class ServletResourceUtils
*/
public static Map<String, String> jsonize(String msgFormat, Object... args)
{
return ImmutableMap.of("error", StringUtils.safeFormat(msgFormat, args));
return ImmutableMap.of("error", StringUtils.nonStrictFormat(msgFormat, args));
}
}

View File

@ -19,6 +19,8 @@
package io.druid.common.utils;
import io.druid.java.util.common.UOE;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.InvocationTargetException;
@ -63,12 +65,7 @@ public class VMUtils
Object maxDirectMemoryObj = vmClass.getMethod("maxDirectMemory").invoke(null);
if (maxDirectMemoryObj == null || !(maxDirectMemoryObj instanceof Number)) {
throw new UnsupportedOperationException(
String.format(
"Cannot determine maxDirectMemory from [%s]",
maxDirectMemoryObj
)
);
throw new UOE("Cannot determine maxDirectMemory from [%s]", maxDirectMemoryObj);
} else {
return ((Number) maxDirectMemoryObj).longValue();
}

View File

@ -19,6 +19,8 @@
package io.druid.math.expr;
import io.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
@ -35,7 +37,7 @@ public class ExprMacroTable
{
this.macroMap = macros.stream().collect(
Collectors.toMap(
m -> m.name().toLowerCase(),
m -> StringUtils.toLowerCase(m.name()),
m -> m
)
);
@ -58,7 +60,7 @@ public class ExprMacroTable
@Nullable
public Expr get(final String functionName, final List<Expr> args)
{
final ExprMacro exprMacro = macroMap.get(functionName.toLowerCase());
final ExprMacro exprMacro = macroMap.get(StringUtils.toLowerCase(functionName));
if (exprMacro == null) {
return null;
}

View File

@ -21,6 +21,7 @@ package io.druid.math.expr;
import com.google.common.base.Strings;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
@ -793,7 +794,7 @@ interface Function
{
ExprType castTo;
try {
castTo = ExprType.valueOf(y.asString().toUpperCase());
castTo = ExprType.valueOf(StringUtils.toUpperCase(y.asString()));
}
catch (IllegalArgumentException e) {
throw new IAE("invalid type '%s'", y.asString());

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.math.expr.antlr.ExprLexer;
import io.druid.math.expr.antlr.ExprParser;
@ -50,7 +51,7 @@ public class Parser
if (!Modifier.isAbstract(clazz.getModifiers()) && Function.class.isAssignableFrom(clazz)) {
try {
Function function = (Function) clazz.newInstance();
functionMap.put(function.name().toLowerCase(), function);
functionMap.put(StringUtils.toLowerCase(function.name()), function);
}
catch (Exception e) {
log.info("failed to instantiate " + clazz.getName() + ".. ignoring", e);
@ -62,7 +63,7 @@ public class Parser
public static Function getFunction(String name)
{
return FUNCTIONS.get(name.toLowerCase());
return FUNCTIONS.get(StringUtils.toLowerCase(name));
}
public static Expr parse(String in, ExprMacroTable macroTable)

View File

@ -19,13 +19,15 @@
package io.druid.metadata;
import io.druid.java.util.common.StringUtils;
public class EntryExistsException extends Exception
{
private final String entryId;
public EntryExistsException(String entryId, Throwable t)
{
super(String.format("Entry already exists: %s", entryId), t);
super(StringUtils.format("Entry already exists: %s", entryId), t);
this.entryId = entryId;
}

View File

@ -20,6 +20,7 @@
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.java.util.common.StringUtils;
/**
*/
@ -61,7 +62,7 @@ public class MetadataStorageConnectorConfig
public String getConnectURI()
{
if (connectURI == null) {
return String.format("jdbc:derby://%s:%s/druid;create=true", host, port);
return StringUtils.format("jdbc:derby://%s:%s/druid;create=true", host, port);
}
return connectURI;
}

View File

@ -22,6 +22,7 @@ package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Maps;
import io.druid.java.util.common.StringUtils;
import java.util.Map;
@ -113,7 +114,7 @@ public class MetadataStorageTablesConfig
if (base == null) {
return null;
}
return String.format("%s_%s", base, defaultSuffix);
return StringUtils.format("%s_%s", base, defaultSuffix);
}
return explicitTableName;

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import io.druid.common.utils.JodaUtils;
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.guava.Comparators;
import io.druid.timeline.partition.ImmutablePartitionHolder;
import io.druid.timeline.partition.PartitionChunk;
@ -415,13 +416,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
// This occurs when restoring segments
timeline.remove(currKey);
} else {
throw new UnsupportedOperationException(
String.format(
"Cannot add overlapping segments [%s and %s] with the same version [%s]",
currKey,
entryInterval,
entry.getVersion()
)
throw new UOE(
"Cannot add overlapping segments [%s and %s] with the same version [%s]",
currKey,
entryInterval,
entry.getVersion()
);
}
}

View File

@ -25,12 +25,13 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.metamx.common.logger.Logger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import twitter4j.ConnectionLifeCycleListener;
import twitter4j.GeoLocation;
import twitter4j.HashtagEntity;
@ -195,13 +196,13 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
@Override
public void onException(Exception ex)
{
ex.printStackTrace();
log.error(ex, "Got exception");
}
@Override
public void onStallWarning(StallWarning warning)
{
System.out.println("Got stall warning:" + warning);
log.warn("Got stall warning: %s", warning);
}
};
@ -306,7 +307,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
long[] lcontrobutors = status.getContributors();
List<String> contributors = new ArrayList<>();
for (long contrib : lcontrobutors) {
contributors.add(String.format("%d", contrib));
contributors.add(StringUtils.format("%d", contrib));
}
theMap.put("contributors", contributors);
@ -346,7 +347,7 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowP
theMap.put("lang", hasUser ? user.getLang() : "");
theMap.put("utc_offset", hasUser ? user.getUtcOffset() : -1); // resolution in seconds, -1 if not available?
theMap.put("statuses_count", hasUser ? user.getStatusesCount() : 0);
theMap.put("user_id", hasUser ? String.format("%d", user.getId()) : "");
theMap.put("user_id", hasUser ? StringUtils.format("%d", user.getId()) : "");
theMap.put("screen_name", hasUser ? user.getScreenName() : "");
theMap.put("location", hasUser ? user.getLocation() : "");
theMap.put("verified", hasUser ? user.isVerified() : "");

View File

@ -36,6 +36,13 @@
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>java-util</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>

View File

@ -20,6 +20,8 @@
package io.druid.extendedset.intset;
import io.druid.java.util.common.StringUtils;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
@ -2276,7 +2278,7 @@ public class ConciseSet extends AbstractIntSet implements java.io.Serializable
if (bit == 0) {
s.append("none");
} else {
s.append(String.format("%4d", bit - 1));
s.append(StringUtils.format("%4d", bit - 1));
}
s.append(')');
}

View File

@ -1,333 +0,0 @@
/*
* (c) 2010 Alessandro Colantonio
* <mailto:colanton@mat.uniroma3.it>
* <http://ricerca.mat.uniroma3.it/users/colanton>
*
* 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 io.druid.extendedset.utilities;
import java.util.Random;
/**
* Population count (a.k.a. Hamming distance) of a bitmap represented by an
* array of <code>int</code>.
* <p>
* Derived from <a
* href="http://dalkescientific.com/writings/diary/popcnt.c">http
* ://dalkescientific.com/writings/diary/popcnt.c</a>
*
* @author Alessandro Colantonio
* @version $Id: BitCount.java 157 2011-11-14 14:25:15Z cocciasik $
*/
public class BitCount
{
/**
* Population count
*
* @param buffer array of <code>int</code>
*
* @return population count
*/
public static int count(int[] buffer)
{
return count(buffer, buffer.length);
}
/**
* Population count
* <p>
* It counts 24 words at a time, then 3 at a time, then 1 at a time
*
* @param buffer array of <code>int</code>
* @param n number of elements of <code>buffer</code> to count
*
* @return population count
*/
public static int count(int[] buffer, int n)
{
final int n1 = n - n % 24;
final int n2 = n - n % 3;
int cnt = 0;
int i;
for (i = 0; i < n1; i += 24) {
cnt += merging3(buffer, i);
}
for (; i < n2; i += 3) {
cnt += merging2(buffer, i);
}
cnt += popcount_fbsd2(buffer, i, n);
return cnt;
}
// used by count()
private static int merging3(int[] buffer, int x)
{
int cnt1;
int cnt2;
int cnt = 0;
for (int i = x; i < x + 24; i += 3) {
cnt1 = buffer[i];
cnt2 = buffer[i + 1];
final int w = buffer[i + 2];
cnt1 = cnt1 - ((cnt1 >>> 1) & 0x55555555) + (w & 0x55555555);
cnt2 = cnt2 - ((cnt2 >>> 1) & 0x55555555) + ((w >>> 1) & 0x55555555);
cnt1 = (cnt1 & 0x33333333) + ((cnt1 >>> 2) & 0x33333333);
cnt1 += (cnt2 & 0x33333333) + ((cnt2 >>> 2) & 0x33333333);
cnt += (cnt1 & 0x0F0F0F0F) + ((cnt1 >>> 4) & 0x0F0F0F0F);
}
cnt = (cnt & 0x00FF00FF) + ((cnt >>> 8) & 0x00FF00FF);
cnt += cnt >>> 16;
return cnt & 0x00000FFFF;
}
// used by count()
private static int merging2(int[] buffer, int x)
{
int cnt1 = buffer[x];
int cnt2 = buffer[x + 1];
final int w = buffer[x + 2];
cnt1 = cnt1 - ((cnt1 >>> 1) & 0x55555555) + (w & 0x55555555);
cnt2 = cnt2 - ((cnt2 >>> 1) & 0x55555555) + ((w >>> 1) & 0x55555555);
cnt1 = (cnt1 & 0x33333333) + ((cnt1 >>> 2) & 0x33333333);
cnt2 = (cnt2 & 0x33333333) + ((cnt2 >>> 2) & 0x33333333);
cnt1 += cnt2;
cnt1 = (cnt1 & 0x0F0F0F0F) + ((cnt1 >>> 4) & 0x0F0F0F0F);
cnt1 += cnt1 >>> 8;
cnt1 += cnt1 >>> 16;
return cnt1 & 0x000000FF;
}
// used by count()
private static int popcount_fbsd2(int[] data, int x, int n)
{
int cnt = 0;
for (; x < n; x++) {
cnt += Integer.bitCount(data[x]);
}
return cnt;
}
/**
* Population count, skipping words at even positions
*
* @param buffer array of <code>int</code>
*
* @return population count
*/
public static int count_2(int[] buffer)
{
return count_2(buffer, buffer.length);
}
/**
* Population count, skipping words at even positions
* <p>
* It counts 24 words at a time, then 3 at a time, then 1 at a time
*
* @param buffer array of <code>int</code>
* @param n number of elements of <code>buffer</code> to count
*
* @return population count
*/
public static int count_2(int[] buffer, int n)
{
final int n1 = n - n % 48;
final int n2 = n - n % 6;
int cnt = 0;
int i;
for (i = 0; i < n1; i += 48) {
cnt += merging3_2(buffer, i);
}
for (; i < n2; i += 6) {
cnt += merging2_2(buffer, i);
}
cnt += popcount_fbsd2_2(buffer, i, n);
return cnt;
}
// used by count_2()
private static int merging3_2(int[] buffer, int x)
{
int cnt1;
int cnt2;
int cnt = 0;
for (int i = x; i < x + 48; i += 6) {
cnt1 = buffer[i + 1];
cnt2 = buffer[i + 3];
final int w = buffer[i + 5];
cnt1 = cnt1 - ((cnt1 >>> 1) & 0x55555555) + (w & 0x55555555);
cnt2 = cnt2 - ((cnt2 >>> 1) & 0x55555555) + ((w >>> 1) & 0x55555555);
cnt1 = (cnt1 & 0x33333333) + ((cnt1 >>> 2) & 0x33333333);
cnt1 += (cnt2 & 0x33333333) + ((cnt2 >>> 2) & 0x33333333);
cnt += (cnt1 & 0x0F0F0F0F) + ((cnt1 >>> 4) & 0x0F0F0F0F);
}
cnt = (cnt & 0x00FF00FF) + ((cnt >>> 8) & 0x00FF00FF);
cnt += cnt >>> 16;
return cnt & 0x00000FFFF;
}
// used by count_2()
private static int merging2_2(int[] buffer, int x)
{
int cnt1 = buffer[x + 1];
int cnt2 = buffer[x + 3];
final int w = buffer[x + 5];
cnt1 = cnt1 - ((cnt1 >>> 1) & 0x55555555) + (w & 0x55555555);
cnt2 = cnt2 - ((cnt2 >>> 1) & 0x55555555) + ((w >>> 1) & 0x55555555);
cnt1 = (cnt1 & 0x33333333) + ((cnt1 >>> 2) & 0x33333333);
cnt2 = (cnt2 & 0x33333333) + ((cnt2 >>> 2) & 0x33333333);
cnt1 += cnt2;
cnt1 = (cnt1 & 0x0F0F0F0F) + ((cnt1 >>> 4) & 0x0F0F0F0F);
cnt1 += cnt1 >>> 8;
cnt1 += cnt1 >>> 16;
return cnt1 & 0x000000FF;
}
// used by count_2()
private static int popcount_fbsd2_2(int[] data, int x, int n)
{
int cnt = 0;
for (x++; x < n; x += 2) {
cnt += Integer.bitCount(data[x]);
}
return cnt;
}
/**
* Test
*
* @param args
*/
public static void main(String[] args)
{
final int trials = 10000;
final int maxLength = 10000;
Random rnd = new Random();
final int seed = rnd.nextInt();
System.out.print("Test correctness... ");
rnd = new Random(seed);
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength)];
for (int j = 0; j < x.length; j++) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
int size1 = 0;
for (int j = 0; j < x.length; j++) {
size1 += Integer.bitCount(x[j]);
}
int size2 = count(x);
if (size1 != size2) {
System.out.println("i = " + i);
System.out.println("ERRORE!");
System.out.println(size1 + ", " + size2);
for (int j = 0; j < x.length; j++) {
System.out.format("x[%d] = %d --> %d\n", j, x[j], Integer.bitCount(x[j]));
}
return;
}
}
System.out.println("done!");
System.out.print("Test correctness II... ");
rnd = new Random(seed);
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength << 1)];
for (int j = 1; j < x.length; j += 2) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
int size1 = 0;
for (int j = 1; j < x.length; j += 2) {
size1 += Integer.bitCount(x[j]);
}
int size2 = count_2(x);
if (size1 != size2) {
System.out.println("i = " + i);
System.out.println("ERRORE!");
System.out.println(size1 + ", " + size2);
for (int j = 1; j < x.length; j += 2) {
System.out.format("x[%d] = %d --> %d\n", j, x[j], Integer.bitCount(x[j]));
}
return;
}
}
System.out.println("done!");
System.out.print("Test time count(): ");
rnd = new Random(seed);
long t = System.currentTimeMillis();
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength)];
for (int j = 0; j < x.length; j++) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
@SuppressWarnings("unused")
int size = 0;
for (int j = 0; j < x.length; j++) {
size += Integer.bitCount(x[j]);
}
}
System.out.println(System.currentTimeMillis() - t);
System.out.print("Test time BitCount.count(): ");
rnd = new Random(seed);
t = System.currentTimeMillis();
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength)];
for (int j = 0; j < x.length; j++) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
count(x);
}
System.out.println(System.currentTimeMillis() - t);
System.out.print("Test II time count(): ");
rnd = new Random(seed);
t = System.currentTimeMillis();
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength << 1)];
for (int j = 1; j < x.length; j += 2) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
@SuppressWarnings("unused")
int size = 0;
for (int j = 1; j < x.length; j += 2) {
size += Integer.bitCount(x[j]);
}
}
System.out.println(System.currentTimeMillis() - t);
System.out.print("Test II time BitCount.count(): ");
rnd = new Random(seed);
t = System.currentTimeMillis();
for (int i = 0; i < trials; i++) {
int[] x = new int[rnd.nextInt(maxLength << 1)];
for (int j = 1; j < x.length; j += 2) {
x[j] = rnd.nextInt(Integer.MAX_VALUE);
}
count_2(x);
}
System.out.println(System.currentTimeMillis() - t);
}
}

View File

@ -17,6 +17,7 @@
package io.druid.extendedset.intset;
import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import junit.framework.Assert;
import org.junit.Test;
@ -1338,7 +1339,7 @@ public class ImmutableConciseSetTest
for (int i = 0; i < length; i++) {
final int n = intIterator.next();
if (i != n) {
Assert.assertEquals(String.format("Failure at bit [%d] on length [%d]", i, length), i, n);
Assert.assertEquals(StringUtils.format("Failure at bit [%d] on length [%d]", i, length), i, n);
}
}
NoSuchElementException ex = null;

View File

@ -26,6 +26,7 @@ import com.metamx.emitter.core.Emitter;
import com.metamx.emitter.core.Event;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.java.util.common.StringUtils;
import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
@ -70,7 +71,7 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements
this.emitterList = emitterList;
this.timelineMetricConverter = config.getDruidToTimelineEventConverter();
this.eventsQueue = new LinkedBlockingQueue<>(config.getMaxQueueSize());
this.collectorURI = String.format(
this.collectorURI = StringUtils.format(
"%s://%s:%s%s",
config.getProtocol(),
config.getHostname(),

View File

@ -26,6 +26,7 @@ import com.google.inject.Inject;
import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
@ -88,8 +89,8 @@ public class AzureDataSegmentPusher implements DataSegmentPusher
final String storageDir = this.getStorageDir(segment);
return ImmutableMap.of(
"index", String.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME),
"descriptor", String.format("%s/%s", storageDir, AzureStorageDruidModule.DESCRIPTOR_FILE_NAME)
"index", StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME),
"descriptor", StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.DESCRIPTOR_FILE_NAME)
);
}

View File

@ -33,6 +33,7 @@ import io.druid.guice.Binders;
import io.druid.guice.JsonConfigProvider;
import io.druid.guice.LazySingleton;
import io.druid.initialization.DruidModule;
import io.druid.java.util.common.StringUtils;
import java.net.URISyntaxException;
import java.security.InvalidKeyException;
@ -103,7 +104,7 @@ public class AzureStorageDruidModule implements DruidModule
throws URISyntaxException, InvalidKeyException
{
CloudStorageAccount account = CloudStorageAccount.parse(
String.format(
StringUtils.format(
STORAGE_CONNECTION_STRING,
config.getProtocol(),
config.getAccount(),

View File

@ -24,7 +24,8 @@ import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import com.microsoft.azure.storage.StorageException;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.tasklogs.TaskLogs;
@ -106,13 +107,13 @@ public class AzureTaskLogs implements TaskLogs {
}
);
} catch (StorageException | URISyntaxException e) {
throw new IOException(String.format("Failed to stream logs from: %s", taskKey), e);
throw new IOE(e, "Failed to stream logs from: %s", taskKey);
}
}
private String getTaskLogKey(String taskid) {
return String.format("%s/%s/log", config.getPrefix(), taskid);
return StringUtils.format("%s/%s/log", config.getPrefix(), taskid);
}
@Override

View File

@ -27,6 +27,7 @@ import com.google.common.io.Files;
import com.microsoft.azure.storage.StorageException;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMockSupport;
@ -116,9 +117,9 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
final String storageDir = pusher.getStorageDir(dataSegment);
Map<String, String> paths = pusher.getAzurePaths(dataSegment);
assertEquals(String.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME), paths.get("index"));
assertEquals(StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME), paths.get("index"));
assertEquals(
String.format("%s/%s", storageDir, AzureStorageDruidModule.DESCRIPTOR_FILE_NAME),
StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.DESCRIPTOR_FILE_NAME),
paths.get("descriptor")
);
}

View File

@ -38,6 +38,7 @@ import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.parsers.ParseException;
@ -556,7 +557,7 @@ public class RocketMQFirehoseFactory implements FirehoseFactory<ByteBufferInputR
}
if (LOGGER.isDebugEnabled() && stringBuilder.length() > 2) {
LOGGER.debug(String.format(
LOGGER.debug(StringUtils.format(
"%s@%s is consuming the following message queues: %s",
defaultMQPullConsumer.getClientIP(),
defaultMQPullConsumer.getInstanceName(),

View File

@ -27,6 +27,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
@ -71,7 +72,7 @@ public class GoogleDataSegmentPusher implements DataSegmentPusher
@Override
public String getPathForHadoop()
{
return String.format("gs://%s/%s", config.getBucket(), config.getPrefix());
return StringUtils.format("gs://%s/%s", config.getBucket(), config.getPrefix());
}
public File createDescriptorFile(final ObjectMapper jsonMapper, final DataSegment segment)

View File

@ -23,13 +23,14 @@ import com.google.api.client.http.InputStreamContent;
import com.google.common.base.Optional;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.logger.Logger;
import io.druid.tasklogs.TaskLogs;
import java.io.File;
import java.io.FileInputStream;
import java.io.InputStream;
import java.io.IOException;
import java.io.InputStream;
public class GoogleTaskLogs implements TaskLogs {
private static final Logger LOG = new Logger(GoogleTaskLogs.class);
@ -93,7 +94,7 @@ public class GoogleTaskLogs implements TaskLogs {
}
);
} catch (IOException e) {
throw new IOException(String.format("Failed to stream logs from: %s", taskKey), e);
throw new IOE(e, "Failed to stream logs from: %s", taskKey);
}
}

View File

@ -32,6 +32,7 @@ import io.druid.data.input.FirehoseFactoryV2;
import io.druid.data.input.FirehoseV2;
import io.druid.data.input.InputRow;
import io.druid.firehose.kafka.KafkaSimpleConsumer.BytesMessageWithOffset;
import io.druid.java.util.common.StringUtils;
import java.io.Closeable;
import java.io.IOException;
@ -323,7 +324,7 @@ public class KafkaEightSimpleConsumerFirehoseFactory implements
}
};
thread.setDaemon(true);
thread.setName(String.format("kafka-%s-%s", topic, partitionId));
thread.setName(StringUtils.format("kafka-%s-%s", topic, partitionId));
thread.start();
}

View File

@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.net.HostAndPort;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.logger.Logger;
import kafka.api.FetchRequest;
@ -96,7 +97,7 @@ public class KafkaSimpleConsumer
this.allBrokers = Collections.unmodifiableList(brokerList);
this.topic = topic;
this.partitionId = partitionId;
this.clientId = String.format("%s_%d_%s", topic, partitionId, clientId);
this.clientId = StringUtils.format("%s_%d_%s", topic, partitionId, clientId);
this.leaderLookupClientId = clientId + "leaderLookup";
this.replicaBrokers = new ArrayList<>();
this.replicaBrokers.addAll(this.allBrokers);

View File

@ -29,7 +29,7 @@ import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import org.apache.commons.lang.StringUtils;
import io.druid.java.util.common.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
@ -107,13 +107,13 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
}
TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeString);
Preconditions.checkArgument(typeInfo instanceof StructTypeInfo,
String.format("typeString should be struct type but not [%s]", typeString));
StringUtils.format("typeString should be struct type but not [%s]", typeString));
Properties table = getTablePropertiesFromStructTypeInfo((StructTypeInfo)typeInfo);
serde.initialize(new Configuration(), table);
try {
oip = (StructObjectInspector) serde.getObjectInspector();
} catch (SerDeException e) {
e.printStackTrace();
throw new RuntimeException(e);
}
}
@ -170,7 +170,8 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
builder.append(parseSpec.getTimestampSpec().getTimestampColumn()).append(":string");
if (parseSpec.getDimensionsSpec().getDimensionNames().size() > 0) {
builder.append(",");
builder.append(StringUtils.join(parseSpec.getDimensionsSpec().getDimensionNames(), ":string,")).append(":string");
builder.append(String.join(":string,", parseSpec.getDimensionsSpec().getDimensionNames()));
builder.append(":string");
}
builder.append(">");
@ -180,17 +181,19 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
public static Properties getTablePropertiesFromStructTypeInfo(StructTypeInfo structTypeInfo)
{
Properties table = new Properties();
table.setProperty("columns", StringUtils.join(structTypeInfo.getAllStructFieldNames(), ","));
table.setProperty("columns.types", StringUtils.join(
Lists.transform(structTypeInfo.getAllStructFieldTypeInfos(),
table.setProperty("columns", String.join(",", structTypeInfo.getAllStructFieldNames()));
table.setProperty("columns.types", String.join(
",",
Lists.transform(
structTypeInfo.getAllStructFieldTypeInfos(),
new Function<TypeInfo, String>() {
@Nullable
@Override
public String apply(@Nullable TypeInfo typeInfo) {
return typeInfo.getTypeName();
}
}),
","
}
)
));
return table;

View File

@ -252,7 +252,7 @@ public class OrcIndexGeneratorJobTest
for (DateTime currTime = interval.getStart(); currTime.isBefore(interval.getEnd()); currTime = currTime.plusDays(1)) {
Integer[][] shardInfo = shardInfoForEachSegment[segmentNum++];
File segmentOutputFolder = new File(
String.format(
StringUtils.format(
"%s/%s/%s_%s/%s",
config.getSchema().getIOConfig().getSegmentOutputPath(),
config.getSchema().getDataSchema().getDataSource(),

View File

@ -36,6 +36,7 @@ import java.util.ArrayList;
import java.util.Calendar;
import java.util.Date;
import java.util.List;
import java.util.Locale;
import java.util.Random;
/**
@ -174,11 +175,11 @@ public class RabbitMQProducerMain
int interval = Integer.parseInt(cmd.getOptionValue("interval", "10"));
int delay = Integer.parseInt(cmd.getOptionValue("delay", "100"));
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss", Locale.ENGLISH);
Date stop = sdf.parse(cmd.getOptionValue("stop", sdf.format(new Date())));
Random r = new Random();
Calendar timer = Calendar.getInstance();
Calendar timer = Calendar.getInstance(Locale.ENGLISH);
timer.setTime(sdf.parse(cmd.getOptionValue("start", "2010-01-01T00:00:00")));
String msg_template = "{\"utcdt\": \"%s\", \"wp\": %d, \"gender\": \"%s\", \"age\": %d}";
@ -188,12 +189,12 @@ public class RabbitMQProducerMain
channel.exchangeDeclare(exchange, type, durable, autoDelete, null);
do{
do {
int wp = (10 + r.nextInt(90)) * 100;
String gender = r.nextBoolean() ? "male" : "female";
int age = 20 + r.nextInt(70);
String line = String.format(msg_template, sdf.format(timer.getTime()), wp, gender, age);
String line = StringUtils.format(msg_template, sdf.format(timer.getTime()), wp, gender, age);
channel.basicPublish(exchange, routingKey, null, StringUtils.toUtf8(line));
@ -202,7 +203,7 @@ public class RabbitMQProducerMain
timer.add(Calendar.SECOND, interval);
Thread.sleep(delay);
}while((!single && stop.after(timer.getTime())));
} while((!single && stop.after(timer.getTime())));
connection.close();
}

View File

@ -18,12 +18,13 @@
*/
package io.druid.metadata.storage.sqlserver;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.regex.Pattern;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.java.util.common.StringUtils;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.SQLMetadataConnector;
import org.apache.commons.dbcp2.BasicDataSource;
import org.skife.jdbi.v2.Binding;
import org.skife.jdbi.v2.ColonPrefixNamedParamStatementRewriter;
@ -35,13 +36,11 @@ import org.skife.jdbi.v2.tweak.RewrittenStatement;
import org.skife.jdbi.v2.tweak.StatementRewriter;
import org.skife.jdbi.v2.util.StringMapper;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.SQLMetadataConnector;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.regex.Pattern;
@SuppressWarnings("nls")
public class SQLServerConnector extends SQLMetadataConnector
@ -236,7 +235,7 @@ public class SQLServerConnector extends SQLMetadataConnector
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(String.format(
handle.createStatement(StringUtils.format(
"MERGE INTO %1$s WITH (UPDLOCK, HOLDLOCK) as target"
+ " USING "
+ " (:key, :value) as source (%2$s, %3$s)"

View File

@ -21,6 +21,7 @@ package io.druid.data.input;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.io.Closeables;
import com.google.common.io.Files;
import io.druid.java.util.common.StringUtils;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.FileReader;
@ -105,7 +106,7 @@ public class AvroHadoopInputRowParserTest
// 1. read avro files into Pig
pigServer = new PigServer(ExecType.LOCAL);
pigServer.registerQuery(
String.format(
StringUtils.format(
"A = LOAD '%s' USING %s;",
someAvroDatumFile,
inputStorage

View File

@ -21,6 +21,8 @@ package io.druid.client.cache;
import com.fasterxml.jackson.annotation.JsonCreator;
import io.druid.concurrent.Execs;
import io.druid.java.util.common.StringUtils;
import java.util.concurrent.Executor;
import java.util.concurrent.ForkJoinPool;
@ -53,6 +55,6 @@ public enum CacheExecutorFactory
@JsonCreator
public static CacheExecutorFactory from(String str)
{
return Enum.valueOf(CacheExecutorFactory.class, str.toUpperCase());
return Enum.valueOf(CacheExecutorFactory.class, StringUtils.toUpperCase(str));
}
}

View File

@ -21,7 +21,6 @@ package io.druid.security.kerberos;
import com.google.common.base.Strings;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.conf.Configuration;
@ -40,6 +39,7 @@ import java.io.IOException;
import java.net.CookieStore;
import java.net.HttpCookie;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.locks.ReentrantLock;
@ -81,7 +81,7 @@ public class DruidKerberosUtil
byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length);
gssContext.dispose();
// Base64 encoded and stringified token for server
return StringUtils.fromUtf8(base64codec.encode(outToken));
return new String(base64codec.encode(outToken), StandardCharsets.US_ASCII);
}
catch (GSSException | IllegalAccessException | NoSuchFieldException | ClassNotFoundException e) {
throw new AuthenticationException(e);

View File

@ -22,12 +22,11 @@ package io.druid.storage.hdfs;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.loading.DataSegmentFinder;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import org.apache.commons.lang.StringUtils;
import io.druid.java.util.common.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
@ -85,8 +84,8 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
final String descriptorParts[] = path.getName().split("_");
if (descriptorParts.length == 2
&& descriptorParts[1].equals("descriptor.json")
&& StringUtils.isNumeric(descriptorParts[0])) {
indexZip = new Path(path.getParent(), String.format("%s_index.zip", descriptorParts[0]));
&& org.apache.commons.lang.StringUtils.isNumeric(descriptorParts[0])) {
indexZip = new Path(path.getParent(), StringUtils.format("%s_index.zip", descriptorParts[0]));
} else {
indexZip = new Path(path.getParent(), "index.zip");
}

View File

@ -82,7 +82,10 @@ public class HdfsDataSegmentKiller implements DataSegmentKiller
segmentPath.toString()
);
}
Path descriptorPath = new Path(segmentPath.getParent(), String.format("%s_descriptor.json", zipParts[0]));
Path descriptorPath = new Path(
segmentPath.getParent(),
io.druid.java.util.common.StringUtils.format("%s_descriptor.json", zipParts[0])
);
//delete partitionNumber_descriptor.json
if (!fs.delete(descriptorPath, false)) {
throw new SegmentLoadingException(

View File

@ -27,6 +27,7 @@ import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.RetryUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.loading.DataSegmentPuller;
@ -318,7 +319,7 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller, URIDataPuller
public String getVersion(URI uri) throws IOException
{
try {
return String.format("%d", buildFileObject(uri, config).getLastModified());
return StringUtils.format("%d", buildFileObject(uri, config).getLastModified());
}
catch (HdfsIOException ex) {
throw ex.getIOException();

View File

@ -26,6 +26,8 @@ import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import io.druid.common.utils.UUIDUtils;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
@ -98,7 +100,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
storageDir
);
Path tmpIndexFile = new Path(String.format(
Path tmpIndexFile = new Path(StringUtils.format(
"%s/%s/%s/%s_index.zip",
fullyQualifiedStorageDirectory,
segment.getDataSource(),
@ -114,13 +116,13 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
final DataSegment dataSegment;
try (FSDataOutputStream out = fs.create(tmpIndexFile)) {
size = CompressionUtils.zip(inDir, out);
final Path outIndexFile = new Path(String.format(
final Path outIndexFile = new Path(StringUtils.format(
"%s/%s/%d_index.zip",
fullyQualifiedStorageDirectory,
storageDir,
segment.getShardSpec().getPartitionNum()
));
final Path outDescriptorFile = new Path(String.format(
final Path outDescriptorFile = new Path(StringUtils.format(
"%s/%s/%d_descriptor.json",
fullyQualifiedStorageDirectory,
storageDir,
@ -133,7 +135,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
final Path tmpDescriptorFile = new Path(
tmpIndexFile.getParent(),
String.format("%s_descriptor.json", dataSegment.getShardSpec().getPartitionNum())
StringUtils.format("%s_descriptor.json", dataSegment.getShardSpec().getPartitionNum())
);
log.info("Creating descriptor file at[%s]", tmpDescriptorFile);
@ -171,11 +173,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
to
);
} else {
throw new IOException(String.format(
"Failed to rename temp Index file[%s] and final segment path[%s] is not present.",
from,
to
));
throw new IOE("Failed to rename temp Index file[%s] and final segment path[%s] is not present.", from, to);
}
}
}
@ -214,7 +212,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
{
return JOINER.join(
segment.getDataSource(),
String.format(
StringUtils.format(
"%s_%s",
segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
@ -226,7 +224,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
@Override
public String makeIndexPathName(DataSegment dataSegment, String indexName)
{
return String.format(
return StringUtils.format(
"./%s/%d_%s",
this.getStorageDir(dataSegment),
dataSegment.getShardSpec().getPartitionNum(),

View File

@ -22,6 +22,7 @@ import com.google.common.base.Optional;
import com.google.common.io.ByteSource;
import com.google.common.io.ByteStreams;
import com.google.inject.Inject;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.logger.Logger;
import io.druid.tasklogs.TaskLogs;
import org.apache.hadoop.conf.Configuration;
@ -135,7 +136,7 @@ public class HdfsTaskLogs implements TaskLogs
if (fs.exists(taskLogDir)) {
if (!fs.isDirectory(taskLogDir)) {
throw new IOException(String.format("taskLogDir [%s] must be a directory.", taskLogDir));
throw new IOE("taskLogDir [%s] must be a directory.", taskLogDir);
}
RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(taskLogDir);

View File

@ -57,7 +57,7 @@ public class HdfsTaskLogsTest
final Map<Long, String> expected = ImmutableMap.of(0L, "blah", 1L, "lah", -2L, "ah", -5L, "blah");
for (Map.Entry<Long, String> entry : expected.entrySet()) {
final String string = readLog(taskLogs, "foo", entry.getKey());
Assert.assertEquals(String.format("Read with offset %,d", entry.getKey()), string, entry.getValue());
Assert.assertEquals(StringUtils.format("Read with offset %,d", entry.getKey()), string, entry.getValue());
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.IOE;
import io.druid.storage.hdfs.HdfsDataSegmentFinder;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NumberedShardSpec;
@ -157,7 +158,7 @@ public class HdfsDataSegmentFinderTest
hdfsTmpDir = File.createTempFile("hdfsDataSource", "dir");
if (!hdfsTmpDir.delete()) {
throw new IOException(String.format("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath()));
throw new IOE("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath());
}
conf = new Configuration(true);
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsTmpDir.getAbsolutePath());

View File

@ -21,6 +21,7 @@ package io.druid.segment.loading;
import com.google.common.io.ByteStreams;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.storage.hdfs.HdfsDataSegmentPuller;
import org.apache.commons.io.FileUtils;
@ -63,7 +64,7 @@ public class HdfsDataSegmentPullerTest
{
hdfsTmpDir = File.createTempFile("hdfsHandlerTest", "dir");
if (!hdfsTmpDir.delete()) {
throw new IOException(String.format("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath()));
throw new IOE("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath());
}
conf = new Configuration(true);
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsTmpDir.getAbsolutePath());

View File

@ -20,6 +20,7 @@
package io.druid.segment.loading;
import com.google.common.io.ByteStreams;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.storage.hdfs.HdfsFileTimestampVersionFinder;
import org.apache.commons.io.FileUtils;
@ -59,7 +60,7 @@ public class HdfsFileTimestampVersionFinderTest
{
hdfsTmpDir = File.createTempFile("hdfsHandlerTest", "dir");
if (!hdfsTmpDir.delete()) {
throw new IOException(String.format("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath()));
throw new IOE("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath());
}
conf = new Configuration(true);
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsTmpDir.getAbsolutePath());

View File

@ -21,6 +21,7 @@ package io.druid.storage.hdfs;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.java.util.common.StringUtils;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.apache.hadoop.conf.Configuration;
@ -173,8 +174,8 @@ public class HdfsDataSegmentKillerTest
Assert.assertTrue(fs.mkdirs(path));
try (FSDataOutputStream os = fs.create(new Path(
path,
String.format("%s_index.zip", partitionNumber)
)); FSDataOutputStream oos = fs.create(new Path(path, String.format("%s_descriptor.json", partitionNumber)))) {
StringUtils.format("%s_index.zip", partitionNumber)
)); FSDataOutputStream oos = fs.create(new Path(path, StringUtils.format("%s_descriptor.json", partitionNumber)))) {
}
}

View File

@ -42,6 +42,7 @@ import io.druid.indexer.HadoopIngestionSpec;
import io.druid.indexer.JobHelper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.jackson.GranularityModule;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.loading.LocalDataSegmentPusher;
import io.druid.segment.loading.LocalDataSegmentPusherConfig;
import io.druid.timeline.DataSegment;
@ -139,7 +140,7 @@ public class HdfsDataSegmentPusherTest
config.setStorageDirectory(
scheme != null
? String.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
? StringUtils.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
: storageDirectory.getAbsolutePath()
);
HdfsDataSegmentPusher pusher = new HdfsDataSegmentPusher(config, conf, new DefaultObjectMapper());
@ -159,7 +160,7 @@ public class HdfsDataSegmentPusherTest
DataSegment segment = pusher.push(segmentDir, segmentToPush);
String indexUri = String.format(
String indexUri = StringUtils.format(
"%s/%s/%d_index.zip",
FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(),
pusher.getStorageDir(segmentToPush),
@ -177,14 +178,14 @@ public class HdfsDataSegmentPusherTest
// rename directory after push
final String segmentPath = pusher.getStorageDir(segment);
File indexFile = new File(String.format(
File indexFile = new File(StringUtils.format(
"%s/%s/%d_index.zip",
storageDirectory,
segmentPath,
segment.getShardSpec().getPartitionNum()
));
Assert.assertTrue(indexFile.exists());
File descriptorFile = new File(String.format(
File descriptorFile = new File(StringUtils.format(
"%s/%s/%d_descriptor.json",
storageDirectory,
segmentPath,
@ -193,7 +194,7 @@ public class HdfsDataSegmentPusherTest
Assert.assertTrue(descriptorFile.exists());
// push twice will fail and temp dir cleaned
File outDir = new File(String.format("%s/%s", config.getStorageDirectory(), segmentPath));
File outDir = new File(StringUtils.format("%s/%s", config.getStorageDirectory(), segmentPath));
outDir.setReadOnly();
try {
pusher.push(segmentDir, segmentToPush);
@ -221,7 +222,7 @@ public class HdfsDataSegmentPusherTest
config.setStorageDirectory(
scheme != null
? String.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
? StringUtils.format("%s://%s", scheme, storageDirectory.getAbsolutePath())
: storageDirectory.getAbsolutePath()
);
HdfsDataSegmentPusher pusher = new HdfsDataSegmentPusher(config, conf, new DefaultObjectMapper());
@ -243,7 +244,7 @@ public class HdfsDataSegmentPusherTest
for (int i = 0; i < numberOfSegments; i++) {
final DataSegment pushedSegment = pusher.push(segmentDir, segments[i]);
String indexUri = String.format(
String indexUri = StringUtils.format(
"%s/%s/%d_index.zip",
FileSystem.newInstance(conf).makeQualified(new Path(config.getStorageDirectory())).toUri().toString(),
pusher.getStorageDir(segments[i]),
@ -261,14 +262,14 @@ public class HdfsDataSegmentPusherTest
// rename directory after push
String segmentPath = pusher.getStorageDir(pushedSegment);
File indexFile = new File(String.format(
File indexFile = new File(StringUtils.format(
"%s/%s/%d_index.zip",
storageDirectory,
segmentPath,
pushedSegment.getShardSpec().getPartitionNum()
));
Assert.assertTrue(indexFile.exists());
File descriptorFile = new File(String.format(
File descriptorFile = new File(StringUtils.format(
"%s/%s/%d_descriptor.json",
storageDirectory,
segmentPath,
@ -290,7 +291,7 @@ public class HdfsDataSegmentPusherTest
// rename directory after push
segmentPath = pusher.getStorageDir(fromDescriptorFileDataSegment);
indexFile = new File(String.format(
indexFile = new File(StringUtils.format(
"%s/%s/%d_index.zip",
storageDirectory,
segmentPath,
@ -300,7 +301,7 @@ public class HdfsDataSegmentPusherTest
// push twice will fail and temp dir cleaned
File outDir = new File(String.format("%s/%s", config.getStorageDirectory(), segmentPath));
File outDir = new File(StringUtils.format("%s/%s", config.getStorageDirectory(), segmentPath));
outDir.setReadOnly();
try {
pusher.push(segmentDir, segments[i]);

View File

@ -21,6 +21,7 @@ package io.druid.query.aggregation.histogram.sql;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import io.druid.java.util.common.StringUtils;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.histogram.ApproximateHistogram;
import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory;
@ -87,7 +88,7 @@ public class QuantileSqlAggregator implements SqlAggregator
}
final AggregatorFactory aggregatorFactory;
final String histogramName = String.format("%s:agg", name);
final String histogramName = StringUtils.format("%s:agg", name);
final RexNode probabilityArg = Expressions.fromFieldAccess(
rowSignature,
project,

View File

@ -23,6 +23,7 @@ import com.google.common.primitives.Floats;
import io.druid.query.aggregation.Histogram;
import java.util.Arrays;
import java.util.Locale;
import java.util.Random;
public class ApproximateHistogramErrorBenchmark
@ -104,10 +105,21 @@ public class ApproximateHistogramErrorBenchmark
errs2[i] = tmp[1];
}
System.out
.format("Number of histograms for folding : %s \n", Arrays.toString(numHistsArray));
System.out.format("Errors for approximate histogram : %s \n", Arrays.toString(errs1));
System.out.format("Errors for approximate histogram, ruleFold : %s \n", Arrays.toString(errs2));
System.out.printf(
Locale.ENGLISH,
"Number of histograms for folding : %s %n",
Arrays.toString(numHistsArray)
);
System.out.printf(
Locale.ENGLISH,
"Errors for approximate histogram : %s %n",
Arrays.toString(errs1)
);
System.out.printf(
Locale.ENGLISH,
"Errors for approximate histogram, ruleFold : %s %n",
Arrays.toString(errs2)
);
}
private float[] getErrors()
@ -181,9 +193,9 @@ public class ApproximateHistogramErrorBenchmark
System.out.println(ah1.toHistogram(breaks));
System.out.println("Approximate Histogram Rule Fold:");
System.out.println(ah2.toHistogram(breaks));
System.out.format("Error for approximate histogram: %s \n", err1);
System.out.format("Error for approximate histogram, ruleFold: %s \n", err2);
System.out.format("Error ratio for AHRF: %s \n", err2 / err1);
System.out.printf(Locale.ENGLISH, "Error for approximate histogram: %f %n", err1);
System.out.printf(Locale.ENGLISH, "Error for approximate histogram, ruleFold: %f %n", err2);
System.out.printf(Locale.ENGLISH, "Error ratio for AHRF: %f %n", err2 / err1);
}
return new float[]{err1, err2, err2 / err1};
}

View File

@ -22,6 +22,7 @@ package io.druid.query.aggregation.histogram;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import io.druid.data.input.Row;
import io.druid.java.util.common.StringUtils;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.dimension.DefaultDimensionSpec;
@ -118,7 +119,7 @@ public class ApproximateHistogramGroupByQueryTest
for (GroupByQueryConfig config : configs) {
final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config);
for (QueryRunner<Row> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
final String testName = String.format(
final String testName = StringUtils.format(
"config=%s, runner=%s",
config.toString(),
runner.toString()

View File

@ -21,6 +21,7 @@ package io.druid.query.aggregation.histogram;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import org.junit.Assert;
import org.junit.Test;
@ -254,7 +255,7 @@ public class ApproximateHistogramTest
tFold += System.nanoTime() - t0;
}
System.out.println(String.format("Average folds per second : %f", (double) count / (double) tFold * 1e9));
System.out.println(StringUtils.format("Average folds per second : %f", (double) count / (double) tFold * 1e9));
}
@Test

View File

@ -299,7 +299,7 @@ public class TestKafkaExtractionCluster
kafkaProducerProperties.putAll(kafkaProperties);
kafkaProducerProperties.put(
"metadata.broker.list",
String.format("127.0.0.1:%d", kafkaServer.socketServer().port())
StringUtils.format("127.0.0.1:%d", kafkaServer.socketServer().port())
);
kafkaProperties.put("request.required.acks", "1");
return kafkaProducerProperties;

View File

@ -52,6 +52,7 @@ import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.TaskResource;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.query.DruidMetrics;
@ -197,7 +198,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
{
super(
id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id,
String.format("%s_%s", TYPE, dataSchema.getDataSource()),
StringUtils.format("%s_%s", TYPE, dataSchema.getDataSource()),
taskResource,
dataSchema.getDataSource(),
context
@ -332,7 +333,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
// Set up sequenceNames.
final Map<Integer, String> sequenceNames = Maps.newHashMap();
for (Integer partitionNum : nextOffsets.keySet()) {
sequenceNames.put(partitionNum, String.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum));
sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum));
}
// Set up committer.
@ -712,7 +713,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
} else if (!endOffsets.keySet().containsAll(offsets.keySet())) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(
String.format(
StringUtils.format(
"Request contains partitions not being handled by this task, my partitions: %s",
endOffsets.keySet()
)
@ -732,7 +733,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(
String.format(
StringUtils.format(
"End offset must be >= current offset for partition [%s] (current: %s)",
entry.getKey(),
nextOffsets.get(entry.getKey())
@ -773,7 +774,7 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
{
if (!(status == Status.PAUSED || status == Status.READING)) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(String.format("Can't pause, task is not in a pausable state (state: [%s])", status))
.entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status))
.build();
}

View File

@ -42,7 +42,9 @@ import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.TaskLocation;
import io.druid.indexing.common.TaskStatus;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.realtime.firehose.ChatHandlerResource;
import org.jboss.netty.channel.ChannelException;
import org.jboss.netty.handler.codec.http.HttpMethod;
@ -111,7 +113,7 @@ public class KafkaIndexTaskClient
this.executorService = MoreExecutors.listeningDecorator(
Execs.multiThreaded(
numThreads,
String.format(
StringUtils.format(
"KafkaIndexTaskClient-%s-%%d",
dataSource
)
@ -174,7 +176,7 @@ public class KafkaIndexTaskClient
id,
HttpMethod.POST,
"pause",
timeout > 0 ? String.format("timeout=%d", timeout) : null,
timeout > 0 ? StringUtils.format("timeout=%d", timeout) : null,
true
);
@ -471,17 +473,17 @@ public class KafkaIndexTaskClient
FullResponseHolder response = null;
Request request = null;
TaskLocation location = TaskLocation.unknown();
String path = String.format("%s/%s/%s", BASE_PATH, id, pathSuffix);
String path = StringUtils.format("%s/%s/%s", BASE_PATH, id, pathSuffix);
Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(id);
if (!status.isPresent() || !status.get().isRunnable()) {
throw new TaskNotRunnableException(String.format("Aborting request because task [%s] is not runnable", id));
throw new TaskNotRunnableException(StringUtils.format("Aborting request because task [%s] is not runnable", id));
}
try {
location = taskInfoProvider.getTaskLocation(id);
if (location.equals(TaskLocation.unknown())) {
throw new NoTaskLocationException(String.format("No TaskLocation available for task [%s]", id));
throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", id));
}
// Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently
@ -514,7 +516,7 @@ public class KafkaIndexTaskClient
} else if (responseCode == 400) { // don't bother retrying if it's a bad request
throw new IAE("Received 400 Bad Request with body: %s", response.getContent());
} else {
throw new IOException(String.format("Received status [%d]", responseCode));
throw new IOE("Received status [%d]", responseCode);
}
}
catch (IOException | ChannelException e) {
@ -544,7 +546,7 @@ public class KafkaIndexTaskClient
String urlForLog = (request != null
? request.getUrl().toString()
: String.format("http://%s:%d%s", location.getHost(), location.getPort(), path));
: StringUtils.format("http://%s:%d%s", location.getHost(), location.getPort(), path));
if (!retry) {
// if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was
// for informational purposes only) so don't log a scary stack trace

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.ImmutableMap;
import io.druid.java.util.common.StringUtils;
import java.util.Map;
import java.util.Objects;
@ -45,7 +46,7 @@ public class KafkaPartitions
for (Map.Entry<Integer, Long> entry : partitionOffsetMap.entrySet()) {
Preconditions.checkArgument(
entry.getValue() >= 0,
String.format(
StringUtils.format(
"partition[%d] offset[%d] invalid",
entry.getKey(),
entry.getValue()

View File

@ -67,6 +67,7 @@ import io.druid.indexing.overlord.supervisor.Supervisor;
import io.druid.indexing.overlord.supervisor.SupervisorReport;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.metadata.EntryExistsException;
import io.druid.server.metrics.DruidMonitorSchedulerConfig;
import org.apache.commons.codec.digest.DigestUtils;
@ -238,7 +239,7 @@ public class KafkaSupervisor implements Supervisor
this.ioConfig = spec.getIoConfig();
this.tuningConfig = spec.getTuningConfig();
this.taskTuningConfig = KafkaTuningConfig.copyOf(this.tuningConfig);
this.supervisorId = String.format("KafkaSupervisor-%s", dataSource);
this.supervisorId = StringUtils.format("KafkaSupervisor-%s", dataSource);
this.exec = Execs.singleThreaded(supervisorId);
this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d");
this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d");
@ -697,7 +698,7 @@ public class KafkaSupervisor implements Supervisor
Map<Integer, Long> startPartitions = taskGroups.get(groupId).partitionOffsets;
for (Map.Entry<Integer, Long> entry : startPartitions.entrySet()) {
sb.append(String.format("+%d(%d)", entry.getKey(), entry.getValue()));
sb.append(StringUtils.format("+%d(%d)", entry.getKey(), entry.getValue()));
}
String partitionOffsetStr = sb.toString().substring(1);
@ -733,7 +734,7 @@ public class KafkaSupervisor implements Supervisor
final Properties props = new Properties();
props.setProperty("metadata.max.age.ms", "10000");
props.setProperty("group.id", String.format("kafka-supervisor-%s", getRandomId()));
props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId()));
props.putAll(ioConfig.getConsumerProperties());

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.StringUtils;
import org.joda.time.Duration;
import org.joda.time.Period;
@ -63,7 +64,7 @@ public class KafkaSupervisorIOConfig
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
Preconditions.checkNotNull(
consumerProperties.get(BOOTSTRAP_SERVERS_KEY),
String.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY)
StringUtils.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY)
);
this.replicas = replicas != null ? replicas : 1;

View File

@ -378,7 +378,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
Capture<Request> captured = Capture.newInstance();
expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK);
expect(responseHolder.getContent()).andReturn(String.format("\"%s\"", status.toString())).anyTimes();
expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes();
expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn(
Futures.immediateFuture(responseHolder)
);
@ -634,7 +634,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "stop")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "stop")));
futures.add(client.stopAsync(TEST_IDS.get(i), false));
}
@ -666,7 +666,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "resume")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "resume")));
futures.add(client.resumeAsync(TEST_IDS.get(i)));
}
@ -699,7 +699,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause")));
futures.add(client.pauseAsync(TEST_IDS.get(i)));
}
@ -732,7 +732,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause?timeout=9")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause?timeout=9")));
futures.add(client.pauseAsync(TEST_IDS.get(i), 9));
}
@ -765,7 +765,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<KafkaIndexTask.Status>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "status")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "status")));
futures.add(client.getStatusAsync(TEST_IDS.get(i)));
}
@ -799,7 +799,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<DateTime>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "time/start")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "time/start")));
futures.add(client.getStartTimeAsync(TEST_IDS.get(i)));
}
@ -832,7 +832,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/current")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/current")));
futures.add(client.getCurrentOffsetsAsync(TEST_IDS.get(i), false));
}
@ -865,7 +865,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end")));
futures.add(client.getEndOffsetsAsync(TEST_IDS.get(i)));
}
@ -898,7 +898,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
List<URL> expectedUrls = Lists.newArrayList();
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end")));
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end")));
futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets));
}
@ -933,7 +933,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
for (int i = 0; i < numRequests; i++) {
expectedUrls.add(
new URL(
String.format(
StringUtils.format(
URL_FORMATTER,
TEST_HOST,
TEST_PORT,

View File

@ -1593,7 +1593,7 @@ public class KafkaIndexTaskTest
private List<String> readSegmentDim1(final SegmentDescriptor descriptor) throws IOException
{
File indexZip = new File(
String.format(
StringUtils.format(
"%s/%s/%s_%s/%s/%d/index.zip",
getSegmentDirectory(),
DATA_SCHEMA.getDataSource(),
@ -1605,7 +1605,7 @@ public class KafkaIndexTaskTest
);
File outputLocation = new File(
directory,
String.format(
StringUtils.format(
"%s_%s_%s_%s",
descriptor.getInterval().getStart(),
descriptor.getInterval().getEnd(),

View File

@ -97,7 +97,6 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import static java.lang.String.format;
import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.anyString;
import static org.easymock.EasyMock.capture;
@ -166,7 +165,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
ImmutableMap.of("num.partitions", String.valueOf(NUM_PARTITIONS))
);
kafkaServer.start();
kafkaHost = String.format("localhost:%d", kafkaServer.getPort());
kafkaHost = StringUtils.format("localhost:%d", kafkaServer.getPort());
dataSchema = getDataSchema(DATASOURCE);
}
@ -1433,7 +1432,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
{
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
taskClient.close();
taskRunner.unregisterListener(String.format("KafkaSupervisor-%s", DATASOURCE));
taskRunner.unregisterListener(StringUtils.format("KafkaSupervisor-%s", DATASOURCE));
replayAll();
supervisor = getSupervisor(1, 1, true, "PT1H", null, false);
@ -1719,7 +1718,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
topic,
i,
null,
StringUtils.toUtf8(format("event-%d", j))
StringUtils.toUtf8(StringUtils.format("event-%d", j))
)
).get();
}
@ -1888,7 +1887,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
@Override
protected String generateSequenceName(int groupId)
{
return String.format("sequenceName-%d", groupId);
return StringUtils.format("sequenceName-%d", groupId);
}
}
}

View File

@ -22,6 +22,7 @@ package io.druid.indexing.kafka.test;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
import io.druid.java.util.common.StringUtils;
import kafka.server.KafkaConfig;
import kafka.server.KafkaServer;
import org.apache.commons.io.FileUtils;
@ -80,7 +81,7 @@ public class TestBroker implements Closeable
final KafkaConfig config = new KafkaConfig(props);
server = new KafkaServer(config, SystemTime.SYSTEM, Some.apply(String.format("TestingBroker[%d]-", id)), List$.MODULE$.empty());
server = new KafkaServer(config, SystemTime.SYSTEM, Some.apply(StringUtils.format("TestingBroker[%d]-", id)), List$.MODULE$.empty());
server.startup();
}
@ -102,7 +103,7 @@ public class TestBroker implements Closeable
public Map<String, String> producerProperties()
{
final Map<String, String> props = Maps.newHashMap();
props.put("bootstrap.servers", String.format("localhost:%d", getPort()));
props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort()));
props.put("key.serializer", ByteArraySerializer.class.getName());
props.put("value.serializer", ByteArraySerializer.class.getName());
props.put("acks", "all");
@ -112,7 +113,7 @@ public class TestBroker implements Closeable
public Map<String, String> consumerProperties()
{
final Map<String, String> props = Maps.newHashMap();
props.put("bootstrap.servers", String.format("localhost:%d", getPort()));
props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort()));
props.put("key.deserializer", ByteArrayDeserializer.class.getName());
props.put("value.deserializer", ByteArrayDeserializer.class.getName());
props.put("group.id", String.valueOf(RANDOM.nextInt()));

View File

@ -80,7 +80,7 @@ public class NamespaceLookupExtractorFactory implements LookupExtractorFactory
Preconditions.checkArgument(this.firstCacheTimeout >= 0);
this.injective = injective;
this.cacheScheduler = cacheScheduler;
this.extractorID = String.format("namespace-factory-%s-%s", extractionNamespace, UUID.randomUUID().toString());
this.extractorID = StringUtils.format("namespace-factory-%s-%s", extractionNamespace, UUID.randomUUID().toString());
this.lookupIntrospectHandler = new NamespaceLookupIntrospectHandler(this);
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import io.druid.java.util.common.StringUtils;
import io.druid.metadata.MetadataStorageConnectorConfig;
import org.joda.time.Period;
@ -108,7 +109,7 @@ public class JdbcExtractionNamespace implements ExtractionNamespace
@Override
public String toString()
{
return String.format(
return StringUtils.format(
"JdbcExtractionNamespace = { connectorConfig = { %s }, table = %s, keyColumn = %s, valueColumn = %s, tsColumn = %s, pollPeriod = %s}",
connectorConfig.toString(),
table,

View File

@ -37,6 +37,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.druid.guice.annotations.Json;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.parsers.CSVParser;
import io.druid.java.util.common.parsers.DelimitedParser;
@ -360,7 +361,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
@Override
public String toString()
{
return String.format(
return StringUtils.format(
"CSVFlatDataParser = { columns = %s, keyColumn = %s, valueColumn = %s }",
Arrays.toString(columns.toArray()),
keyColumn,
@ -502,7 +503,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
@Override
public String toString()
{
return String.format(
return StringUtils.format(
"TSVFlatDataParser = { columns = %s, delimiter = '%s', listDelimiter = '%s',keyColumn = %s, valueColumn = %s }",
Arrays.toString(columns.toArray()),
delimiter,
@ -580,7 +581,7 @@ public class UriExtractionNamespace implements ExtractionNamespace
@Override
public String toString()
{
return String.format(
return StringUtils.format(
"JSONFlatDataParser = { keyFieldName = %s, valueFieldName = %s }",
keyFieldName,
valueFieldName

View File

@ -21,6 +21,7 @@ package io.druid.server.lookup.namespace;
import io.druid.common.utils.JodaUtils;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.lookup.namespace.CacheGenerator;
import io.druid.query.lookup.namespace.JdbcExtractionNamespace;
@ -78,7 +79,7 @@ public final class JdbcCacheGenerator implements CacheGenerator<JdbcExtractionNa
public List<Pair<String, String>> withHandle(Handle handle) throws Exception
{
final String query;
query = String.format(
query = StringUtils.format(
"SELECT %s, %s FROM %s",
keyColumn,
valueColumn,
@ -109,7 +110,7 @@ public final class JdbcCacheGenerator implements CacheGenerator<JdbcExtractionNa
if (lastDBUpdate != null) {
newVersion = lastDBUpdate.toString();
} else {
newVersion = String.format("%d", dbQueryStart);
newVersion = StringUtils.format("%d", dbQueryStart);
}
final CacheScheduler.VersionedCache versionedCache = scheduler.createVersionedCache(entryId, newVersion);
try {
@ -165,7 +166,7 @@ public final class JdbcCacheGenerator implements CacheGenerator<JdbcExtractionNa
@Override
public Timestamp withHandle(Handle handle) throws Exception
{
final String query = String.format(
final String query = StringUtils.format(
"SELECT MAX(%s) FROM %s",
tsColumn, table
);

View File

@ -26,6 +26,7 @@ import io.druid.data.input.MapPopulator;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.RetryUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.lookup.namespace.CacheGenerator;
import io.druid.query.lookup.namespace.UriExtractionNamespace;
@ -101,7 +102,7 @@ public final class UriCacheGenerator implements CacheGenerator<UriExtractionName
if (uri == null) {
throw new FileNotFoundException(
String.format(
StringUtils.format(
"Could not find match for pattern `%s` in [%s] for %s",
versionRegex,
originalUri,

View File

@ -26,6 +26,7 @@ import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.guice.LazySingleton;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.lookup.namespace.CacheGenerator;
import io.druid.query.lookup.namespace.ExtractionNamespace;
@ -151,7 +152,7 @@ public final class CacheScheduler
{
try {
this.namespace = namespace;
this.asString = String.format("namespace [%s] : %s", namespace, super.toString());
this.asString = StringUtils.format("namespace [%s] : %s", namespace, super.toString());
this.updaterFuture = schedule(namespace);
this.entryCleaner = createCleaner(entry);
this.cacheGenerator = cacheGenerator;

View File

@ -31,6 +31,7 @@ import io.druid.guice.GuiceAnnotationIntrospector;
import io.druid.guice.GuiceInjectableValues;
import io.druid.guice.annotations.Json;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.StringUtils;
import org.junit.Assert;
import org.junit.Test;
@ -180,7 +181,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%s\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
valueField
@ -204,7 +205,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
valueField
@ -227,7 +228,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
valueField
@ -250,7 +251,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
valueField
@ -273,7 +274,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%sDFSDFDS\":\"B\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
valueField
@ -387,7 +388,7 @@ public class UriExtractionNamespaceTest
final String keyField = "keyField";
final String valueField = "valueField";
final int n = 341879;
final String nString = String.format("%d", n);
final String nString = StringUtils.format("%d", n);
UriExtractionNamespace.JSONFlatDataParser parser = new UriExtractionNamespace.JSONFlatDataParser(
new ObjectMapper(),
keyField,
@ -398,7 +399,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", nString),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%s\":\"B\", \"%s\":\"%d\", \"FOO\":\"BAR\"}",
keyField,
valueField,
@ -411,7 +412,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of(nString, "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%s\":\"%d\", \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
n,
@ -424,7 +425,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of("B", nString),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%s\":\"B\", \"%s\":%d, \"FOO\":\"BAR\"}",
keyField,
valueField,
@ -437,7 +438,7 @@ public class UriExtractionNamespaceTest
ImmutableMap.of(nString, "C"),
parser.getParser()
.parse(
String.format(
StringUtils.format(
"{\"%s\":%d, \"%s\":\"C\", \"FOO\":\"BAR\"}",
keyField,
n,
@ -454,10 +455,10 @@ public class UriExtractionNamespaceTest
registerTypes(new DefaultObjectMapper())
);
final int n = 341879;
final String nString = String.format("%d", n);
final String nString = StringUtils.format("%d", n);
Assert.assertEquals(
ImmutableMap.of("key", nString),
parser.getParser().parse(String.format("{\"key\":%d}", n))
parser.getParser().parse(StringUtils.format("{\"key\":%d}", n))
);
}
}

View File

@ -27,6 +27,7 @@ import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.concurrent.Execs;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.lifecycle.Lifecycle;
import io.druid.query.lookup.namespace.CacheGenerator;
import io.druid.query.lookup.namespace.ExtractionNamespace;
@ -207,7 +208,7 @@ public class CacheSchedulerTest
long minEnd = start + ((repeatCount - 1) * delay);
long end = System.currentTimeMillis();
Assert.assertTrue(
String.format(
StringUtils.format(
"Didn't wait long enough between runs. Expected more than %d was %d",
minEnd - start,
end - start

View File

@ -27,6 +27,7 @@ import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.concurrent.Execs;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.lifecycle.Lifecycle;
import io.druid.java.util.common.logger.Logger;
@ -122,7 +123,7 @@ public class JdbcExtractionNamespaceTest
Assert.assertEquals(
0,
handle.createStatement(
String.format(
StringUtils.format(
"CREATE TABLE %s (%s TIMESTAMP, %s VARCHAR(64), %s VARCHAR(64))",
tableName,
tsColumn_,
@ -131,7 +132,7 @@ public class JdbcExtractionNamespaceTest
)
).setQueryTimeout(1).execute()
);
handle.createStatement(String.format("TRUNCATE TABLE %s", tableName)).setQueryTimeout(1).execute();
handle.createStatement(StringUtils.format("TRUNCATE TABLE %s", tableName)).setQueryTimeout(1).execute();
handle.commit();
closer.register(new Closeable()
{
@ -327,16 +328,16 @@ public class JdbcExtractionNamespaceTest
final String query;
if (tsColumn == null) {
handle.createStatement(
String.format("DELETE FROM %s WHERE %s='%s'", tableName, keyName, key)
StringUtils.format("DELETE FROM %s WHERE %s='%s'", tableName, keyName, key)
).setQueryTimeout(1).execute();
query = String.format(
query = StringUtils.format(
"INSERT INTO %s (%s, %s) VALUES ('%s', '%s')",
tableName,
keyName, valName,
key, val
);
} else {
query = String.format(
query = StringUtils.format(
"INSERT INTO %s (%s, %s, %s) VALUES ('%s', '%s', '%s')",
tableName,
tsColumn, keyName, valName,

View File

@ -24,6 +24,7 @@ import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import io.druid.java.util.common.StringUtils;
import org.mapdb.DB;
import org.mapdb.DBMaker;
import org.mapdb.HTreeMap;
@ -49,8 +50,8 @@ public class OffHeapPollingCache<K, V> implements PollingCache<K, V>
public OffHeapPollingCache(final Iterable<Map.Entry<K, V>> entries)
{
synchronized (started) {
this.cacheName = String.format("cache-%s", UUID.randomUUID());
this.reverseCacheName = String.format("reverseCache-%s", UUID.randomUUID());
this.cacheName = StringUtils.format("cache-%s", UUID.randomUUID());
this.reverseCacheName = StringUtils.format("reverseCache-%s", UUID.randomUUID());
mapCache = DB.createHashMap(cacheName).make();
reverseCache = DB.createHashMap(reverseCacheName).make();
ImmutableSet.Builder<V> setOfValuesBuilder = ImmutableSet.builder();

View File

@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.server.lookup.DataFetcher;
@ -75,19 +76,19 @@ public class JdbcDataFetcher implements DataFetcher<String, String>
this.keyColumn = Preconditions.checkNotNull(keyColumn, "keyColumn");
this.valueColumn = Preconditions.checkNotNull(valueColumn, "valueColumn");
this.fetchAllQuery = String.format(
this.fetchAllQuery = StringUtils.format(
"SELECT %s, %s FROM %s",
this.keyColumn,
this.valueColumn,
this.table
);
this.fetchQuery = String.format(
this.fetchQuery = StringUtils.format(
"SELECT %s FROM %s WHERE %s = :val",
this.valueColumn,
this.table,
this.keyColumn
);
this.reverseFetchQuery = String.format(
this.reverseFetchQuery = StringUtils.format(
"SELECT %s FROM %s WHERE %s = :val",
this.keyColumn,
this.table,

View File

@ -22,6 +22,7 @@ package io.druid.server.lookup.jdbc;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.StringUtils;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.TestDerbyConnector;
import io.druid.server.lookup.DataFetcher;
@ -66,7 +67,7 @@ public class JdbcDataFetcherTest
Assert.assertEquals(
0,
handle.createStatement(
String.format(
StringUtils.format(
"CREATE TABLE %s (%s VARCHAR(64), %s VARCHAR(64))",
tableName,
keyColumn,
@ -74,7 +75,7 @@ public class JdbcDataFetcherTest
)
).setQueryTimeout(1).execute()
);
handle.createStatement(String.format("TRUNCATE TABLE %s", tableName)).setQueryTimeout(1).execute();
handle.createStatement(StringUtils.format("TRUNCATE TABLE %s", tableName)).setQueryTimeout(1).execute();
for (Map.Entry<String, String> entry : lookupMap.entrySet()) {
insertValues(entry.getKey(), entry.getValue(), handle);
@ -167,9 +168,9 @@ public class JdbcDataFetcherTest
{
final String query;
handle.createStatement(
String.format("DELETE FROM %s WHERE %s='%s'", tableName, keyColumn, key)
StringUtils.format("DELETE FROM %s WHERE %s='%s'", tableName, keyColumn, key)
).setQueryTimeout(1).execute();
query = String.format(
query = StringUtils.format(
"INSERT INTO %s (%s, %s) VALUES ('%s', '%s')",
tableName,
keyColumn, valueColumn,

View File

@ -25,6 +25,7 @@ import com.google.inject.Inject;
import com.mysql.jdbc.exceptions.MySQLTransientException;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
@ -135,7 +136,7 @@ public class MySQLConnector extends SQLMetadataConnector
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
StringUtils.format(
"INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value) ON DUPLICATE KEY UPDATE %3$s = :value",
tableName,
keyColumn,

View File

@ -22,6 +22,7 @@ package io.druid.metadata.storage.postgresql;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
@ -126,7 +127,7 @@ public class PostgreSQLConnector extends SQLMetadataConnector
{
if (canUpsert(handle)) {
handle.createStatement(
String.format(
StringUtils.format(
"INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value) ON CONFLICT (%2$s) DO UPDATE SET %3$s = EXCLUDED.%3$s",
tableName,
keyColumn,
@ -138,7 +139,7 @@ public class PostgreSQLConnector extends SQLMetadataConnector
.execute();
} else {
handle.createStatement(
String.format(
StringUtils.format(
"BEGIN;\n" +
"LOCK TABLE %1$s IN SHARE ROW EXCLUSIVE MODE;\n" +
"WITH upsert AS (UPDATE %1$s SET %3$s=:value WHERE %2$s=:key RETURNING *)\n" +

View File

@ -32,6 +32,7 @@ import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.ParseSpec;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.java.util.common.parsers.Parser;
@ -139,7 +140,7 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
Descriptor desc = dynamicSchema.getMessageDescriptor(messageType);
if (desc == null) {
throw new ParseException(
String.format(
StringUtils.format(
"Protobuf message type %s not found in the specified descriptor. Available messages types are %s",
protoMessageType,
messageTypes

View File

@ -136,7 +136,7 @@ public class S3DataSegmentMover implements DataSegmentMover
s3Object.getStorageClass().equals(S3Object.STORAGE_CLASS_GLACIER)) {
log.warn("Cannot move file[s3://%s/%s] of storage class glacier, skipping.", s3Bucket, s3Path);
} else {
final String copyMsg = StringUtils.safeFormat(
final String copyMsg = StringUtils.format(
"[s3://%s/%s] to [s3://%s/%s]", s3Bucket,
s3Path,
targetS3Bucket,

View File

@ -28,6 +28,7 @@ import com.google.inject.Inject;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.FileUtils;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.MapUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.UOE;
@ -100,7 +101,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
}
}
catch (ServiceException e) {
throw new IOException(StringUtils.safeFormat("Could not load S3 URI [%s]", uri), e);
throw new IOE(e, "Could not load S3 URI [%s]", uri);
}
}
@ -178,7 +179,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
try {
org.apache.commons.io.FileUtils.forceMkdir(outDir);
final URI uri = URI.create(String.format("s3://%s/%s", s3Coords.bucket, s3Coords.path));
final URI uri = URI.create(StringUtils.format("s3://%s/%s", s3Coords.bucket, s3Coords.path));
final ByteSource byteSource = new ByteSource()
{
@Override
@ -250,7 +251,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
return buildFileObject(uri, s3Client).openInputStream();
}
catch (ServiceException e) {
throw new IOException(String.format("Could not load URI [%s]", uri.toString()), e);
throw new IOE(e, "Could not load URI [%s]", uri);
}
}
@ -292,15 +293,12 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
{
try {
final FileObject object = buildFileObject(uri, s3Client);
return String.format("%d", object.getLastModified());
return StringUtils.format("%d", object.getLastModified());
}
catch (ServiceException e) {
if (S3Utils.isServiceExceptionRecoverable(e)) {
// The recoverable logic is always true for IOException, so we want to only pass IOException if it is recoverable
throw new IOException(
String.format("Could not fetch last modified timestamp from URI [%s]", uri.toString()),
e
);
throw new IOE(e, "Could not fetch last modified timestamp from URI [%s]", uri);
} else {
throw Throwables.propagate(e);
}
@ -366,7 +364,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller, URIDataPuller
@Override
public String toString()
{
return String.format("s3://%s/%s", bucket, path);
return StringUtils.format("s3://%s/%s", bucket, path);
}
}
}

View File

@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import com.metamx.emitter.EmittingLogger;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.SegmentUtils;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.timeline.DataSegment;
@ -66,9 +67,9 @@ public class S3DataSegmentPusher implements DataSegmentPusher
public String getPathForHadoop()
{
if (config.isUseS3aSchema()) {
return String.format("s3a://%s/%s", config.getBucket(), config.getBaseKey());
return StringUtils.format("s3a://%s/%s", config.getBucket(), config.getBaseKey());
}
return String.format("s3n://%s/%s", config.getBucket(), config.getBaseKey());
return StringUtils.format("s3n://%s/%s", config.getBucket(), config.getBaseKey());
}
@Deprecated

View File

@ -24,6 +24,8 @@ import com.google.common.base.Throwables;
import com.google.common.io.ByteSource;
import com.google.inject.Inject;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.tasklogs.TaskLogs;
import org.jets3t.service.ServiceException;
@ -103,7 +105,7 @@ public class S3TaskLogs implements TaskLogs
|| "NoSuchBucket".equals(e.getErrorCode())) {
return Optional.absent();
} else {
throw new IOException(String.format("Failed to stream logs from: %s", taskKey), e);
throw new IOE(e, "Failed to stream logs from: %s", taskKey);
}
}
}
@ -137,7 +139,7 @@ public class S3TaskLogs implements TaskLogs
private String getTaskLogKey(String taskid)
{
return String.format("%s/%s/log", config.getS3Prefix(), taskid);
return StringUtils.format("%s/%s/log", config.getS3Prefix(), taskid);
}
@Override

Some files were not shown because too many files have changed in this diff Show More