Update all String conversions to and from byte[] to use the java-util StringUtils functions

* Speedup of GroupBy with javaScript filters by ~10%
* Requires https://github.com/metamx/java-util/pull/15
This commit is contained in:
Charles Allen 2014-12-09 17:03:01 -08:00
parent c0469c99d6
commit b1b5c9099e
53 changed files with 123 additions and 70 deletions

View File

@ -27,6 +27,7 @@ import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Floats; import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.Aggregators;
@ -153,7 +154,7 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory
return ah; return ah;
} else if (object instanceof String) { } else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8)); byte[] bytes = Base64.decodeBase64(StringUtils.toUtf8((String) object));
final ApproximateHistogram ah = ApproximateHistogram.fromBytes(bytes); final ApproximateHistogram ah = ApproximateHistogram.fromBytes(bytes);
ah.setLowerLimit(lowerLimit); ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit); ah.setUpperLimit(upperLimit);
@ -216,7 +217,7 @@ public class ApproximateHistogramAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2) return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(fieldNameBytes) .put(fieldNameBytes)

View File

@ -26,6 +26,7 @@ import com.google.common.base.Charsets;
import com.google.common.primitives.Floats; import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.BufferAggregator;
@ -139,7 +140,7 @@ public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHis
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2) return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(fieldNameBytes) .put(fieldNameBytes)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.common.StringUtils;
import com.rabbitmq.client.AMQP; import com.rabbitmq.client.AMQP;
import com.rabbitmq.client.Channel; import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection; import com.rabbitmq.client.Connection;
@ -234,7 +235,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory<StringInputRowPa
return null; return null;
} }
return stringParser.parse(new String(delivery.getBody(), Charsets.UTF_8)); return stringParser.parse(StringUtils.fromUtf8(delivery.getBody()));
} }
@Override @Override

View File

@ -20,6 +20,7 @@
package io.druid.indexer; package io.druid.indexer;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
@ -76,8 +77,8 @@ public class SortableBytes
public String toString() public String toString()
{ {
return "SortableBytes{" + return "SortableBytes{" +
"groupKey='" + new String(groupKey, Charsets.UTF_8) + '\'' + "groupKey='" + StringUtils.fromUtf8(groupKey) + '\'' +
", sortKey='" + new String(sortKey, Charsets.UTF_8) + '\'' + ", sortKey='" + StringUtils.fromUtf8(sortKey) + '\'' +
'}'; '}';
} }

View File

@ -20,6 +20,7 @@
package io.druid.indexer; package io.druid.indexer;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.io.WritableComparator;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -108,11 +109,11 @@ public class SortableBytesTest
private byte[] toBytes(String string) private byte[] toBytes(String string)
{ {
return string.getBytes(Charsets.UTF_8); return StringUtils.toUtf8(string);
} }
private String fromBytes(byte[] bytes) private String fromBytes(byte[] bytes)
{ {
return new String(bytes, Charsets.UTF_8); return StringUtils.fromUtf8(bytes);
} }
} }

View File

@ -22,6 +22,7 @@ package io.druid.indexing.overlord.autoscaling.ec2;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
public class StringEC2UserData implements EC2UserData<StringEC2UserData> public class StringEC2UserData implements EC2UserData<StringEC2UserData>
@ -75,7 +76,7 @@ public class StringEC2UserData implements EC2UserData<StringEC2UserData>
} else { } else {
finalData = data; finalData = data;
} }
return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8)); return Base64.encodeBase64String(StringUtils.toUtf8(finalData));
} }
@Override @Override

View File

@ -20,6 +20,7 @@
package io.druid.indexing.overlord; package io.druid.indexing.overlord;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.TestUtils;
import io.druid.indexing.overlord.autoscaling.ec2.EC2UserData; import io.druid.indexing.overlord.autoscaling.ec2.EC2UserData;
import io.druid.indexing.overlord.autoscaling.ec2.GalaxyEC2UserData; import io.druid.indexing.overlord.autoscaling.ec2.GalaxyEC2UserData;
@ -52,11 +53,11 @@ public class WorkerSetupDataTest
Assert.assertEquals("hey :ver:", userData.getData()); Assert.assertEquals("hey :ver:", userData.getData());
Assert.assertEquals("1234", userData.getVersion()); Assert.assertEquals("1234", userData.getVersion());
Assert.assertEquals( Assert.assertEquals(
Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)), Base64.encodeBase64String(StringUtils.toUtf8("hey 1234")),
userData.getUserDataBase64() userData.getUserDataBase64()
); );
Assert.assertEquals( Assert.assertEquals(
Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)), Base64.encodeBase64String(StringUtils.toUtf8("hey xyz")),
userData.withVersion("xyz").getUserDataBase64() userData.withVersion("xyz").getUserDataBase64()
); );
} }

View File

@ -40,7 +40,7 @@
<properties> <properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.26.9</metamx.java-util.version> <metamx.java-util.version>0.26.11</metamx.java-util.version>
<apache.curator.version>2.7.0</apache.curator.version> <apache.curator.version>2.7.0</apache.curator.version>
<jetty.version>9.2.5.v20141112</jetty.version> <jetty.version>9.2.5.v20141112</jetty.version>
<druid.api.version>0.3.1</druid.api.version> <druid.api.version>0.3.1</druid.api.version>

View File

@ -22,6 +22,7 @@ package io.druid.granularity;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.joda.time.Chronology; import org.joda.time.Chronology;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
@ -326,7 +327,7 @@ public class PeriodGranularity extends BaseQueryGranularity
@Override @Override
public byte[] cacheKey() public byte[] cacheKey()
{ {
return (period.toString() + ":" + chronology.getZone().toString() + ":" + origin).getBytes(Charsets.UTF_8); return StringUtils.toUtf8(period.toString() + ":" + chronology.getZone().toString() + ":" + origin);
} }
@Override @Override

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles; import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -130,7 +131,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.primitives.Floats; import com.google.common.primitives.Floats;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
@ -114,7 +115,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory
} else if (object instanceof ByteBuffer) { } else if (object instanceof ByteBuffer) {
return Histogram.fromBytes((ByteBuffer) object); return Histogram.fromBytes((ByteBuffer) object);
} else if (object instanceof String) { } else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8)); byte[] bytes = Base64.decodeBase64(StringUtils.toUtf8((String) object));
return Histogram.fromBytes(bytes); return Histogram.fromBytes(bytes);
} }
return object; return object;
@ -154,7 +155,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -26,6 +26,7 @@ import com.google.common.base.Joiner;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.primitives.Doubles; import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ObjectColumnSelector; import io.druid.segment.ObjectColumnSelector;
import org.mozilla.javascript.Context; import org.mozilla.javascript.Context;
@ -211,8 +212,8 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory
{ {
try { try {
MessageDigest md = MessageDigest.getInstance("SHA-1"); MessageDigest md = MessageDigest.getInstance("SHA-1");
byte[] fieldNameBytes = Joiner.on(",").join(fieldNames).getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(Joiner.on(",").join(fieldNames));
byte[] sha1 = md.digest((fnAggregate + fnReset + fnCombine).getBytes(Charsets.UTF_8)); byte[] sha1 = md.digest(StringUtils.toUtf8(fnAggregate + fnReset + fnCombine));
return ByteBuffer.allocate(1 + fieldNameBytes.length + sha1.length) return ByteBuffer.allocate(1 + fieldNameBytes.length + sha1.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -126,7 +127,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles; import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -127,7 +128,7 @@ public class MaxAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles; import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -127,7 +128,7 @@ public class MinAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -27,6 +27,7 @@ import com.google.common.base.Joiner;
import com.google.common.base.Predicates; import com.google.common.base.Predicates;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.Aggregators;
@ -171,7 +172,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory
return HyperLogLogCollector.makeCollector((ByteBuffer) object); return HyperLogLogCollector.makeCollector((ByteBuffer) object);
} else if (object instanceof String) { } else if (object instanceof String) {
return HyperLogLogCollector.makeCollector( return HyperLogLogCollector.makeCollector(
ByteBuffer.wrap(Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8))) ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)))
); );
} }
return object; return object;
@ -212,7 +213,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = Joiner.on("\u0001").join(fieldNames).getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(Joiner.on("\u0001").join(fieldNames));
return ByteBuffer.allocate(2 + fieldNameBytes.length) return ByteBuffer.allocate(2 + fieldNameBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.Aggregators;
@ -148,7 +149,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
return HyperLogLogCollector.makeCollector((ByteBuffer) object); return HyperLogLogCollector.makeCollector((ByteBuffer) object);
} else if (object instanceof String) { } else if (object instanceof String) {
return HyperLogLogCollector.makeCollector( return HyperLogLogCollector.makeCollector(
ByteBuffer.wrap(Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8))) ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)))
); );
} }
return object; return object;
@ -183,7 +184,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array();
} }

View File

@ -22,6 +22,7 @@ package io.druid.query.aggregation.hyperloglog;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.hash.HashFunction; import com.google.common.hash.HashFunction;
import com.metamx.common.StringUtils;
import io.druid.data.input.InputRow; import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexed;
@ -93,7 +94,7 @@ public class HyperUniquesSerde extends ComplexMetricSerde
for (String dimensionValue : dimValues) { for (String dimensionValue : dimValues) {
collector.add( collector.add(
hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes() hashFn.hashBytes(StringUtils.toUtf8(dimensionValue)).asBytes()
); );
} }
return collector; return collector;

View File

@ -22,6 +22,7 @@ package io.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -69,7 +70,7 @@ public class DefaultDimensionSpec implements DimensionSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimensionBytes = StringUtils.toUtf8(dimension);
return ByteBuffer.allocate(1 + dimensionBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -22,6 +22,7 @@ package io.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -74,7 +75,7 @@ public class ExtractionDimensionSpec implements DimensionSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] dimExtractionFnBytes = dimExtractionFn.getCacheKey(); byte[] dimExtractionFnBytes = dimExtractionFn.getCacheKey();
return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.metamx.common.StringUtils;
import org.mozilla.javascript.Context; import org.mozilla.javascript.Context;
import org.mozilla.javascript.ContextFactory; import org.mozilla.javascript.ContextFactory;
import org.mozilla.javascript.ScriptableObject; import org.mozilla.javascript.ScriptableObject;
@ -81,7 +82,7 @@ public class JavascriptDimExtractionFn implements DimExtractionFn
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] bytes = function.getBytes(Charsets.UTF_8); byte[] bytes = StringUtils.toUtf8(function);
return ByteBuffer.allocate(1 + bytes.length) return ByteBuffer.allocate(1 + bytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(bytes) .put(bytes)

View File

@ -22,6 +22,7 @@ package io.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.regex.Matcher; import java.util.regex.Matcher;
@ -48,7 +49,7 @@ public class PartialDimExtractionFn implements DimExtractionFn
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] exprBytes = expr.getBytes(Charsets.UTF_8); byte[] exprBytes = StringUtils.toUtf8(expr);
return ByteBuffer.allocate(1 + exprBytes.length) return ByteBuffer.allocate(1 + exprBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(exprBytes) .put(exprBytes)

View File

@ -22,6 +22,7 @@ package io.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.regex.Matcher; import java.util.regex.Matcher;
@ -48,7 +49,7 @@ public class RegexDimExtractionFn implements DimExtractionFn
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] exprBytes = expr.getBytes(Charsets.UTF_8); byte[] exprBytes = StringUtils.toUtf8(expr);
return ByteBuffer.allocate(1 + exprBytes.length) return ByteBuffer.allocate(1 + exprBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(exprBytes) .put(exprBytes)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.ibm.icu.text.SimpleDateFormat; import com.ibm.icu.text.SimpleDateFormat;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.text.ParseException; import java.text.ParseException;
@ -56,7 +57,7 @@ public class TimeDimExtractionFn implements DimExtractionFn
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] timeFormatBytes = timeFormat.getBytes(Charsets.UTF_8); byte[] timeFormatBytes = StringUtils.toUtf8(timeFormat);
return ByteBuffer.allocate(1 + timeFormatBytes.length) return ByteBuffer.allocate(1 + timeFormatBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(timeFormatBytes) .put(timeFormatBytes)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.DimExtractionFn; import io.druid.query.extraction.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -72,8 +73,8 @@ public class ExtractionDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] valueBytes = value.getBytes(Charsets.UTF_8); byte[] valueBytes = StringUtils.toUtf8(value);
return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length)
.put(DimFilterCacheHelper.EXTRACTION_CACHE_ID) .put(DimFilterCacheHelper.EXTRACTION_CACHE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -58,8 +59,8 @@ public class JavaScriptDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
final byte[] functionBytes = function.getBytes(Charsets.UTF_8); final byte[] functionBytes = StringUtils.toUtf8(function);
return ByteBuffer.allocate(1 + dimensionBytes.length + functionBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + functionBytes.length)
.put(DimFilterCacheHelper.JAVASCRIPT_CACHE_ID) .put(DimFilterCacheHelper.JAVASCRIPT_CACHE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -60,8 +61,8 @@ public class RegexDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
final byte[] patternBytes = pattern.getBytes(Charsets.UTF_8); final byte[] patternBytes = StringUtils.toUtf8(pattern);
return ByteBuffer.allocate(1 + dimensionBytes.length + patternBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + patternBytes.length)
.put(DimFilterCacheHelper.REGEX_CACHE_ID) .put(DimFilterCacheHelper.REGEX_CACHE_ID)

View File

@ -22,6 +22,7 @@ package io.druid.query.filter;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.common.StringUtils;
import io.druid.query.search.search.SearchQuerySpec; import io.druid.query.search.search.SearchQuerySpec;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -60,7 +61,7 @@ public class SearchQueryDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
final byte[] queryBytes = query.getCacheKey(); final byte[] queryBytes = query.getCacheKey();
return ByteBuffer.allocate(1 + dimensionBytes.length + queryBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + queryBytes.length)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -48,8 +49,8 @@ public class SelectorDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] valueBytes = (value == null) ? new byte[]{} : value.getBytes(Charsets.UTF_8); byte[] valueBytes = (value == null) ? new byte[]{} : StringUtils.toUtf8(value);
return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length) return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length)
.put(DimFilterCacheHelper.SELECTOR_CACHE_ID) .put(DimFilterCacheHelper.SELECTOR_CACHE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.collections.spatial.search.Bound; import com.metamx.collections.spatial.search.Bound;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -49,7 +50,7 @@ public class SpatialDimFilter implements DimFilter
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] dimBytes = dimension.getBytes(Charsets.UTF_8); byte[] dimBytes = StringUtils.toUtf8(dimension);
byte[] boundBytes = bound.getCacheKey(); byte[] boundBytes = bound.getCacheKey();
return ByteBuffer.allocate(1 + dimBytes.length + boundBytes.length) return ByteBuffer.allocate(1 + dimBytes.length + boundBytes.length)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.primitives.Bytes; import com.google.common.primitives.Bytes;
import com.metamx.common.StringUtils;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -72,7 +73,7 @@ public class EqualToHavingSpec implements HavingSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = Bytes.toArray(Arrays.asList(value));
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.primitives.Bytes; import com.google.common.primitives.Bytes;
import com.metamx.common.StringUtils;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -72,7 +73,7 @@ public class GreaterThanHavingSpec implements HavingSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = Bytes.toArray(Arrays.asList(value));
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)

View File

@ -22,6 +22,7 @@ package io.druid.query.groupby.having;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.primitives.Bytes; import com.google.common.primitives.Bytes;
import com.metamx.common.StringUtils;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -70,7 +71,7 @@ public class LessThanHavingSpec implements HavingSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = aggregationName.getBytes(Charsets.UTF_8); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = Bytes.toArray(Arrays.asList(value));
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)

View File

@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.IAE; import com.metamx.common.IAE;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.StringUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -183,8 +184,8 @@ public class OrderByColumnSpec
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8); final byte[] dimensionBytes = StringUtils.toUtf8(dimension);
final byte[] directionBytes = direction.name().getBytes(Charsets.UTF_8); final byte[] directionBytes = StringUtils.toUtf8(direction.name());
return ByteBuffer.allocate(dimensionBytes.length + directionBytes.length) return ByteBuffer.allocate(dimensionBytes.length + directionBytes.length)
.put(dimensionBytes) .put(dimensionBytes)

View File

@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.common.StringUtils;
import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.ColumnAnalysis;
import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndex;
import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.BitmapIndex;
@ -122,7 +123,7 @@ public class SegmentAnalyzer
String value = bitmapIndex.getValue(i); String value = bitmapIndex.getValue(i);
if (value != null) { if (value != null) {
size += value.getBytes(Charsets.UTF_8).length * bitmapIndex.getBitmap(value).size(); size += StringUtils.toUtf8(value).length * bitmapIndex.getBitmap(value).size();
} }
} }

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
@ -64,7 +65,7 @@ public class ListColumnIncluderator implements ColumnIncluderator
List<byte[]> columns = Lists.newArrayListWithExpectedSize(this.columns.size()); List<byte[]> columns = Lists.newArrayListWithExpectedSize(this.columns.size());
for (String column : this.columns) { for (String column : this.columns) {
final byte[] bytes = column.getBytes(Charsets.UTF_8); final byte[] bytes = StringUtils.toUtf8(column);
columns.add(bytes); columns.add(bytes);
size += bytes.length; size += bytes.length;
} }

View File

@ -35,6 +35,7 @@ import com.metamx.common.guava.MergeSequence;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.guava.nary.BinaryFn;
import com.metamx.common.StringUtils;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.collections.OrderedMergeSequence; import io.druid.collections.OrderedMergeSequence;
import io.druid.query.CacheStrategy; import io.druid.query.CacheStrategy;
@ -159,7 +160,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResul
int dimensionsBytesSize = 0; int dimensionsBytesSize = 0;
int index = 0; int index = 0;
for (String dimension : dimensions) { for (String dimension : dimensions) {
dimensionsBytes[index] = dimension.getBytes(Charsets.UTF_8); dimensionsBytes[index] = StringUtils.toUtf8(dimension);
dimensionsBytesSize += dimensionsBytes[index].length; dimensionsBytesSize += dimensionsBytes[index].length;
++index; ++index;
} }

View File

@ -22,6 +22,7 @@ package io.druid.query.search.search;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
@ -66,7 +67,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
int valuesBytesSize = 0; int valuesBytesSize = 0;
int index = 0; int index = 0;
for (String value : values) { for (String value : values) {
valuesBytes[index] = value.getBytes(Charsets.UTF_8); valuesBytes[index] = StringUtils.toUtf8(value);
valuesBytesSize += valuesBytes[index].length; valuesBytesSize += valuesBytes[index].length;
++index; ++index;
} }

View File

@ -22,6 +22,7 @@ package io.druid.query.search.search;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -59,7 +60,7 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] valueBytes = value.getBytes(Charsets.UTF_8); byte[] valueBytes = StringUtils.toUtf8(value);
return ByteBuffer.allocate(1 + valueBytes.length) return ByteBuffer.allocate(1 + valueBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
@ -66,7 +67,7 @@ public class PagingSpec
int pagingKeysSize = 0; int pagingKeysSize = 0;
int pagingValuesSize = 0; int pagingValuesSize = 0;
for (Map.Entry<String, Integer> entry : pagingIdentifiers.entrySet()) { for (Map.Entry<String, Integer> entry : pagingIdentifiers.entrySet()) {
pagingKeys[index] = entry.getKey().getBytes(Charsets.UTF_8); pagingKeys[index] = StringUtils.toUtf8(entry.getKey());
pagingValues[index] = ByteBuffer.allocate(Ints.BYTES).putInt(entry.getValue()).array(); pagingValues[index] = ByteBuffer.allocate(Ints.BYTES).putInt(entry.getValue()).array();
pagingKeysSize += pagingKeys[index].length; pagingKeysSize += pagingKeys[index].length;
pagingValuesSize += Ints.BYTES; pagingValuesSize += Ints.BYTES;

View File

@ -30,6 +30,7 @@ import com.google.inject.Inject;
import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.MergeSequence;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.guava.nary.BinaryFn;
import com.metamx.common.StringUtils;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.collections.OrderedMergeSequence; import io.druid.collections.OrderedMergeSequence;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
@ -160,7 +161,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
int dimensionsBytesSize = 0; int dimensionsBytesSize = 0;
int index = 0; int index = 0;
for (String dimension : dimensions) { for (String dimension : dimensions) {
dimensionsBytes[index] = dimension.getBytes(Charsets.UTF_8); dimensionsBytes[index] = StringUtils.toUtf8(dimension);
dimensionsBytesSize += dimensionsBytes[index].length; dimensionsBytesSize += dimensionsBytes[index].length;
++index; ++index;
} }
@ -174,7 +175,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest<Result<SelectResul
int metricBytesSize = 0; int metricBytesSize = 0;
index = 0; index = 0;
for (String metric : metrics) { for (String metric : metrics) {
metricBytes[index] = metric.getBytes(Charsets.UTF_8); metricBytes[index] = StringUtils.toUtf8(metric);
metricBytesSize += metricBytes[index].length; metricBytesSize += metricBytes[index].length;
++index; ++index;
} }

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.common.StringUtils;
import io.druid.common.utils.JodaUtils; import io.druid.common.utils.JodaUtils;
import io.druid.query.BaseQuery; import io.druid.query.BaseQuery;
import io.druid.query.DataSource; import io.druid.query.DataSource;
@ -125,7 +126,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] boundBytes = bound.getBytes(Charsets.UTF_8); final byte[] boundBytes = StringUtils.toUtf8(bound);
return ByteBuffer.allocate(1 + boundBytes.length) return ByteBuffer.allocate(1 + boundBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(boundBytes) .put(boundBytes)

View File

@ -23,6 +23,7 @@ import com.davekoelle.alphanum.AlphanumComparator;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.PostAggregator;
@ -54,7 +55,7 @@ public class AlphaNumericTopNMetricSpec extends LexicographicTopNMetricSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] previousStopBytes = previousStop.getBytes(Charsets.UTF_8); byte[] previousStopBytes = StringUtils.toUtf8(previousStop);
return ByteBuffer.allocate(1 + previousStopBytes.length) return ByteBuffer.allocate(1 + previousStopBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.primitives.UnsignedBytes; import com.google.common.primitives.UnsignedBytes;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.DimensionSpec;
@ -50,7 +51,10 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec
if (s2 == null) { if (s2 == null) {
return 1; return 1;
} }
return UnsignedBytes.lexicographicalComparator().compare(s.getBytes(Charsets.UTF_8), s2.getBytes(Charsets.UTF_8)); return UnsignedBytes.lexicographicalComparator().compare(
StringUtils.toUtf8(s),
StringUtils.toUtf8(s2)
);
} }
}; };
@ -105,7 +109,7 @@ public class LexicographicTopNMetricSpec implements TopNMetricSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] previousStopBytes = previousStop == null ? new byte[]{} : previousStop.getBytes(Charsets.UTF_8); byte[] previousStopBytes = previousStop == null ? new byte[]{} : StringUtils.toUtf8(previousStop);
return ByteBuffer.allocate(1 + previousStopBytes.length) return ByteBuffer.allocate(1 + previousStopBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -132,7 +132,7 @@ public class NumericTopNMetricSpec implements TopNMetricSpec
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
byte[] metricBytes = metric.getBytes(Charsets.UTF_8); byte[] metricBytes = com.metamx.common.StringUtils.toUtf8(metric);
return ByteBuffer.allocate(1 + metricBytes.length) return ByteBuffer.allocate(1 + metricBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)

View File

@ -20,6 +20,7 @@
package io.druid.segment.data; package io.druid.segment.data;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.common.IAE; import com.metamx.common.IAE;
@ -30,6 +31,7 @@ import com.metamx.common.logger.Logger;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel; import java.nio.channels.WritableByteChannel;
import java.util.Arrays; import java.util.Arrays;
@ -377,9 +379,7 @@ public class GenericIndexed<T> implements Indexed<T>, Closeable
@Override @Override
public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) public String fromByteBuffer(final ByteBuffer buffer, final int numBytes)
{ {
final byte[] bytes = new byte[numBytes]; return com.metamx.common.StringUtils.fromUtf8(buffer, numBytes);
buffer.get(bytes);
return new String(bytes, Charsets.UTF_8);
} }
@Override @Override
@ -388,7 +388,7 @@ public class GenericIndexed<T> implements Indexed<T>, Closeable
if (val == null) { if (val == null) {
return new byte[]{}; return new byte[]{};
} }
return val.getBytes(Charsets.UTF_8); return com.metamx.common.StringUtils.toUtf8(val);
} }
@Override @Override

View File

@ -40,7 +40,7 @@ public class CacheUtil
) )
{ {
final Interval segmentQueryInterval = descriptor.getInterval(); final Interval segmentQueryInterval = descriptor.getInterval();
final byte[] versionBytes = descriptor.getVersion().getBytes(Charsets.UTF_8); final byte[] versionBytes = com.metamx.common.StringUtils.toUtf8(descriptor.getVersion());
return new Cache.NamedKey( return new Cache.NamedKey(
segmentIdentifier, ByteBuffer segmentIdentifier, ByteBuffer

View File

@ -28,6 +28,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.MapMaker; import com.google.common.collect.MapMaker;
import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.StringUtils;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import io.druid.concurrent.Execs; import io.druid.concurrent.Execs;
import io.druid.curator.inventory.CuratorInventoryManager; import io.druid.curator.inventory.CuratorInventoryManager;
@ -114,6 +115,7 @@ public abstract class ServerInventoryView<InventoryType> implements ServerView,
return jsonMapper.readValue(bytes, typeReference); return jsonMapper.readValue(bytes, typeReference);
} }
catch (IOException e) { catch (IOException e) {
CharBuffer.wrap(StringUtils.fromUtf8(bytes).toCharArray());
CharBuffer charBuffer = Charsets.UTF_8.decode(ByteBuffer.wrap(bytes)); CharBuffer charBuffer = Charsets.UTF_8.decode(ByteBuffer.wrap(bytes));
log.error(e, "Could not parse json: %s", charBuffer.toString()); log.error(e, "Could not parse json: %s", charBuffer.toString());
throw Throwables.propagate(e); throw Throwables.propagate(e);

View File

@ -22,6 +22,7 @@ package io.druid.client.cache;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.metamx.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
@ -54,7 +55,7 @@ public interface Cache
} }
public byte[] toByteArray() { public byte[] toByteArray() {
final byte[] nsBytes = this.namespace.getBytes(Charsets.UTF_8); final byte[] nsBytes = StringUtils.toUtf8(this.namespace);
return ByteBuffer.allocate(Ints.BYTES + nsBytes.length + this.key.length) return ByteBuffer.allocate(Ints.BYTES + nsBytes.length + this.key.length)
.putInt(nsBytes.length) .putInt(nsBytes.length)
.put(nsBytes) .put(nsBytes)

View File

@ -32,6 +32,7 @@ import com.google.inject.Module;
import com.google.inject.util.Modules; import com.google.inject.util.Modules;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.common.StringUtils;
import io.druid.curator.CuratorModule; import io.druid.curator.CuratorModule;
import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.DiscoveryModule;
import io.druid.guice.AWSModule; import io.druid.guice.AWSModule;
@ -312,7 +313,7 @@ public class Initialization
} }
} }
, false, Charsets.UTF_8.name()) , false, StringUtils.UTF8_STRING)
); );
return new DefaultTeslaAether( return new DefaultTeslaAether(
config.getLocalRepository(), config.getLocalRepository(),

View File

@ -29,6 +29,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.common.Pair; import com.metamx.common.Pair;
import com.metamx.common.RetryUtils; import com.metamx.common.RetryUtils;
import com.metamx.common.StringUtils;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.FoldController;
@ -398,7 +399,7 @@ public class SQLMetadataStorageActionHandler<EntryType, StatusType, LogType, Loc
catch (IOException e) { catch (IOException e) {
log.makeAlert(e, "Failed to deserialize log") log.makeAlert(e, "Failed to deserialize log")
.addData("entryId", entryId) .addData("entryId", entryId)
.addData("payload", new String(bytes, Charsets.UTF_8)) .addData("payload", StringUtils.fromUtf8(bytes))
.emit(); .emit();
throw new SQLException(e); throw new SQLException(e);
} }
@ -445,7 +446,7 @@ public class SQLMetadataStorageActionHandler<EntryType, StatusType, LogType, Loc
log.makeAlert(e, "Failed to deserialize " + lockType.getType()) log.makeAlert(e, "Failed to deserialize " + lockType.getType())
.addData("id", r.getLong("id")) .addData("id", r.getLong("id"))
.addData( .addData(
"lockPayload", new String(r.getBytes("lock_payload"), Charsets.UTF_8) "lockPayload", StringUtils.fromUtf8(r.getBytes("lock_payload"))
) )
.emit(); .emit();
throw new SQLException(e); throw new SQLException(e);

View File

@ -33,6 +33,7 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import com.metamx.common.guava.Yielder; import com.metamx.common.guava.Yielder;
import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingAccumulator;
import com.metamx.common.StringUtils;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Json;

View File

@ -28,6 +28,7 @@ import com.google.common.base.Joiner;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.StringUtils;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.Druids; import io.druid.query.Druids;
@ -145,7 +146,7 @@ public class SQLRunner
URL url = new URL(String.format("http://%s/druid/v2/?pretty", hostname)); URL url = new URL(String.format("http://%s/druid/v2/?pretty", hostname));
final URLConnection urlConnection = url.openConnection(); final URLConnection urlConnection = url.openConnection();
urlConnection.addRequestProperty("content-type", MediaType.APPLICATION_JSON); urlConnection.addRequestProperty("content-type", MediaType.APPLICATION_JSON);
urlConnection.getOutputStream().write(queryStr.getBytes(Charsets.UTF_8)); urlConnection.getOutputStream().write(StringUtils.toUtf8(queryStr));
BufferedReader stdInput = new BufferedReader(new InputStreamReader(urlConnection.getInputStream(), Charsets.UTF_8)); BufferedReader stdInput = new BufferedReader(new InputStreamReader(urlConnection.getInputStream(), Charsets.UTF_8));
Object res = objectMapper.readValue(stdInput, typeRef); Object res = objectMapper.readValue(stdInput, typeRef);