Skip to content

Commit

Permalink
Update all String conversions to and from byte[] to use the java-util…
Browse files Browse the repository at this point in the history
… StringUtils functions

* Speedup of GroupBy with javaScript filters by ~10%
* Requires metamx/java-util#15
  • Loading branch information
drcrallen committed Jan 5, 2015
1 parent c0469c9 commit b1b5c90
Show file tree
Hide file tree
Showing 53 changed files with 123 additions and 70 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.google.common.base.Preconditions;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators;
Expand Down Expand Up @@ -153,7 +154,7 @@ public Object deserialize(Object object)

return ah;
} 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);
ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit);
Expand Down Expand Up @@ -216,7 +217,7 @@ public List<String> requiredFields()
@Override
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)
.put(CACHE_TYPE_ID)
.put(fieldNameBytes)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.metamx.common.IAE;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator;
Expand Down Expand Up @@ -139,7 +140,7 @@ public AggregatorFactory getCombiningFactory()
@Override
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)
.put(CACHE_TYPE_ID)
.put(fieldNameBytes)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.logger.Logger;
import com.metamx.common.StringUtils;
import com.rabbitmq.client.AMQP;
import com.rabbitmq.client.Channel;
import com.rabbitmq.client.Connection;
Expand Down Expand Up @@ -234,7 +235,7 @@ public InputRow nextRow()
return null;
}

return stringParser.parse(new String(delivery.getBody(), Charsets.UTF_8));
return stringParser.parse(StringUtils.fromUtf8(delivery.getBody()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package io.druid.indexer;

import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.Job;
Expand Down Expand Up @@ -76,8 +77,8 @@ public BytesWritable toBytesWritable()
public String toString()
{
return "SortableBytes{" +
"groupKey='" + new String(groupKey, Charsets.UTF_8) + '\'' +
", sortKey='" + new String(sortKey, Charsets.UTF_8) + '\'' +
"groupKey='" + StringUtils.fromUtf8(groupKey) + '\'' +
", sortKey='" + StringUtils.fromUtf8(sortKey) + '\'' +
'}';
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package io.druid.indexer;

import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.hadoop.io.WritableComparator;
import org.junit.Assert;
import org.junit.Test;
Expand Down Expand Up @@ -108,11 +109,11 @@ private byte[] getByteArrayBytes(SortableBytes thingie1) throws IOException

private byte[] toBytes(String string)
{
return string.getBytes(Charsets.UTF_8);
return StringUtils.toUtf8(string);
}

private String fromBytes(byte[] bytes)
{
return new String(bytes, Charsets.UTF_8);
return StringUtils.fromUtf8(bytes);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.apache.commons.codec.binary.Base64;

public class StringEC2UserData implements EC2UserData<StringEC2UserData>
Expand Down Expand Up @@ -75,7 +76,7 @@ public String getUserDataBase64()
} else {
finalData = data;
}
return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8));
return Base64.encodeBase64String(StringUtils.toUtf8(finalData));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package io.druid.indexing.overlord;

import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.overlord.autoscaling.ec2.EC2UserData;
import io.druid.indexing.overlord.autoscaling.ec2.GalaxyEC2UserData;
Expand Down Expand Up @@ -52,11 +53,11 @@ public void testStringEC2UserDataSerde() throws IOException
Assert.assertEquals("hey :ver:", userData.getData());
Assert.assertEquals("1234", userData.getVersion());
Assert.assertEquals(
Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)),
Base64.encodeBase64String(StringUtils.toUtf8("hey 1234")),
userData.getUserDataBase64()
);
Assert.assertEquals(
Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)),
Base64.encodeBase64String(StringUtils.toUtf8("hey xyz")),
userData.withVersion("xyz").getUserDataBase64()
);
}
Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@

<properties>
<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>
<jetty.version>9.2.5.v20141112</jetty.version>
<druid.api.version>0.3.1</druid.api.version>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import org.joda.time.Chronology;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
Expand Down Expand Up @@ -326,7 +327,7 @@ private long truncateMillisPeriod(final long t)
@Override
public byte[] cacheKey()
{
return (period.toString() + ":" + chronology.getZone().toString() + ":" + origin).getBytes(Charsets.UTF_8);
return StringUtils.toUtf8(period.toString() + ":" + chronology.getZone().toString() + ":" + origin);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -130,7 +131,7 @@ public List<String> requiredFields()
@Override
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import com.google.common.collect.Lists;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Longs;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;
import org.apache.commons.codec.binary.Base64;

Expand Down Expand Up @@ -114,7 +115,7 @@ public Object deserialize(Object object)
} else if (object instanceof ByteBuffer) {
return Histogram.fromBytes((ByteBuffer) object);
} 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 object;
Expand Down Expand Up @@ -154,7 +155,7 @@ public List<String> requiredFields()
@Override
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();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ObjectColumnSelector;
import org.mozilla.javascript.Context;
Expand Down Expand Up @@ -211,8 +212,8 @@ public byte[] getCacheKey()
{
try {
MessageDigest md = MessageDigest.getInstance("SHA-1");
byte[] fieldNameBytes = Joiner.on(",").join(fieldNames).getBytes(Charsets.UTF_8);
byte[] sha1 = md.digest((fnAggregate + fnReset + fnCombine).getBytes(Charsets.UTF_8));
byte[] fieldNameBytes = StringUtils.toUtf8(Joiner.on(",").join(fieldNames));
byte[] sha1 = md.digest(StringUtils.toUtf8(fnAggregate + fnReset + fnCombine));

return ByteBuffer.allocate(1 + fieldNameBytes.length + sha1.length)
.put(CACHE_TYPE_ID)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -126,7 +127,7 @@ public List<String> requiredFields()
@Override
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -127,7 +128,7 @@ public List<String> requiredFields()
@Override
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Doubles;
import com.metamx.common.StringUtils;
import io.druid.segment.ColumnSelectorFactory;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -127,7 +128,7 @@ public List<String> requiredFields()
@Override
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.google.common.base.Predicates;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators;
Expand Down Expand Up @@ -171,7 +172,7 @@ public Object deserialize(Object object)
return HyperLogLogCollector.makeCollector((ByteBuffer) object);
} else if (object instanceof String) {
return HyperLogLogCollector.makeCollector(
ByteBuffer.wrap(Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8)))
ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)))
);
}
return object;
Expand Down Expand Up @@ -212,7 +213,7 @@ public boolean isByRow()
@Override
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)
.put(CACHE_TYPE_ID)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.IAE;
import com.metamx.common.StringUtils;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.Aggregators;
Expand Down Expand Up @@ -148,7 +149,7 @@ public Object deserialize(Object object)
return HyperLogLogCollector.makeCollector((ByteBuffer) object);
} else if (object instanceof String) {
return HyperLogLogCollector.makeCollector(
ByteBuffer.wrap(Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8)))
ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)))
);
}
return object;
Expand Down Expand Up @@ -183,7 +184,7 @@ public String getFieldName()
@Override
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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.google.common.base.Charsets;
import com.google.common.collect.Ordering;
import com.google.common.hash.HashFunction;
import com.metamx.common.StringUtils;
import io.druid.data.input.InputRow;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
Expand Down Expand Up @@ -93,7 +94,7 @@ public HyperLogLogCollector extractValue(InputRow inputRow, String metricName)

for (String dimensionValue : dimValues) {
collector.add(
hashFn.hashBytes(dimensionValue.getBytes(Charsets.UTF_8)).asBytes()
hashFn.hashBytes(StringUtils.toUtf8(dimensionValue)).asBytes()
);
}
return collector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.DimExtractionFn;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -69,7 +70,7 @@ public DimExtractionFn getDimExtractionFn()
@Override
public byte[] getCacheKey()
{
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8);
byte[] dimensionBytes = StringUtils.toUtf8(dimension);

return ByteBuffer.allocate(1 + dimensionBytes.length)
.put(CACHE_TYPE_ID)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.StringUtils;
import io.druid.query.extraction.DimExtractionFn;

import java.nio.ByteBuffer;
Expand Down Expand Up @@ -74,7 +75,7 @@ public DimExtractionFn getDimExtractionFn()
@Override
public byte[] getCacheKey()
{
byte[] dimensionBytes = dimension.getBytes(Charsets.UTF_8);
byte[] dimensionBytes = StringUtils.toUtf8(dimension);
byte[] dimExtractionFnBytes = dimExtractionFn.getCacheKey();

return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.google.common.base.Function;
import com.metamx.common.StringUtils;
import org.mozilla.javascript.Context;
import org.mozilla.javascript.ContextFactory;
import org.mozilla.javascript.ScriptableObject;
Expand Down Expand Up @@ -81,7 +82,7 @@ public String getFunction()
@Override
public byte[] getCacheKey()
{
byte[] bytes = function.getBytes(Charsets.UTF_8);
byte[] bytes = StringUtils.toUtf8(function);
return ByteBuffer.allocate(1 + bytes.length)
.put(CACHE_TYPE_ID)
.put(bytes)
Expand Down
Loading

0 comments on commit b1b5c90

Please sign in to comment.