For performance reasons, use `java.util.Base64` instead of Base64 in Apache Commons Codec and Guava (#6913)

* * Add few methods about base64 into StringUtils

* Use `java.util.Base64` instead of others

* Add org.apache.commons.codec.binary.Base64 & com.google.common.io.BaseEncoding into druid-forbidden-apis

* Rename encodeBase64String & decodeBase64String

* Update druid-forbidden-apis
This commit is contained in:
Benedict Jin 2019-01-26 09:32:29 +08:00 committed by Clint Wylie
parent 8492d94f59
commit 72a571fbf7
32 changed files with 100 additions and 72 deletions

View File

@ -33,4 +33,8 @@ java.util.LinkedList @ Use ArrayList or ArrayDeque instead
java.util.Random#<init>() @ Use ThreadLocalRandom.current() or the constructor with a seed (the latter in tests only!)
java.lang.Math#random() @ Use ThreadLocalRandom.current()
java.util.regex.Pattern#matches(java.lang.String,java.lang.CharSequence) @ Use String.startsWith(), endsWith(), contains(), or compile and cache a Pattern explicitly
org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead
org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead
@defaultMessage For performance reasons, use the utf8Base64 / encodeBase64 / encodeBase64String / decodeBase64 / decodeBase64String methods in StringUtils
org.apache.commons.codec.binary.Base64
com.google.common.io.BaseEncoding.base64

View File

@ -43,6 +43,8 @@ public class StringUtils
@Deprecated // Charset parameters to String are currently slower than the charset's string name
public static final Charset UTF8_CHARSET = StandardCharsets.UTF_8;
public static final String UTF8_STRING = StandardCharsets.UTF_8.toString();
private static final Base64.Encoder BASE64_ENCODER = Base64.getEncoder();
private static final Base64.Decoder BASE64_DECODER = Base64.getDecoder();
// should be used only for estimation
// returns the same result with StringUtils.fromUtf8(value).length for valid string values
@ -309,11 +311,54 @@ public class StringUtils
* Convert an input to base 64 and return the utf8 string of that byte array
*
* @param input The string to convert to base64
*
* @return the base64 of the input in string form
*/
public static String utf8Base64(String input)
{
return fromUtf8(Base64.getEncoder().encode(toUtf8(input)));
return fromUtf8(encodeBase64(toUtf8(input)));
}
/**
* Convert an input byte array into a newly-allocated byte array using the {@link Base64} encoding scheme
*
* @param input The byte array to convert to base64
* @return the base64 of the input in byte array form
*/
public static byte[] encodeBase64(byte[] input)
{
return BASE64_ENCODER.encode(input);
}
/**
* Convert an input byte array into a string using the {@link Base64} encoding scheme
*
* @param input The byte array to convert to base64
* @return the base64 of the input in string form
*/
public static String encodeBase64String(byte[] input)
{
return BASE64_ENCODER.encodeToString(input);
}
/**
* Decode an input byte array using the {@link Base64} encoding scheme and return a newly-allocated byte array
*
* @param input The byte array to decode from base64
* @return a newly-allocated byte array
*/
public static byte[] decodeBase64(byte[] input)
{
return BASE64_DECODER.decode(input);
}
/**
* Decode an input string using the {@link Base64} encoding scheme and return a newly-allocated byte array
*
* @param input The string to decode from base64
* @return a newly-allocated byte array
*/
public static byte[] decodeBase64String(String input)
{
return BASE64_DECODER.decode(input);
}
}

View File

@ -47,7 +47,6 @@ import java.net.URL;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayDeque;
import java.util.Base64;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
@ -747,7 +746,7 @@ public class HttpPostEmitter implements Flushable, Closeable, Emitter
final String[] parts = config.getBasicAuthentication().split(":", 2);
final String user = parts[0];
final String password = parts.length > 1 ? parts[1] : "";
String encoded = Base64.getEncoder().encodeToString((user + ':' + password).getBytes(StandardCharsets.UTF_8));
String encoded = StringUtils.encodeBase64String((user + ':' + password).getBytes(StandardCharsets.UTF_8));
request.setHeader(HttpHeaders.Names.AUTHORIZATION, "Basic " + encoded);
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.java.util.emitter.core;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.io.BaseEncoding;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http.DefaultHttpResponse;
import io.netty.handler.codec.http.HttpHeaders;
@ -454,7 +453,7 @@ public class EmitterTest
request.getHeaders().get(HttpHeaders.Names.CONTENT_TYPE)
);
Assert.assertEquals(
"Basic " + BaseEncoding.base64().encode(StringUtils.toUtf8("foo:bar")),
"Basic " + StringUtils.encodeBase64String(StringUtils.toUtf8("foo:bar")),
request.getHeaders().get(HttpHeaders.Names.AUTHORIZATION)
);
Assert.assertEquals(

View File

@ -20,7 +20,7 @@
package org.apache.druid.data.input.thrift;
import com.google.common.base.Preconditions;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.thrift.TBase;
import org.apache.thrift.TDeserializer;
import org.apache.thrift.TException;
@ -85,7 +85,7 @@ public class ThriftDeserialization
return EMPTY_BYTES;
}
final byte last = src[src.length - 1];
return (0 == last || '}' == last) ? src : Base64.decodeBase64(src);
return (0 == last || '}' == last) ? src : StringUtils.decodeBase64(src);
}
/**

View File

@ -21,7 +21,6 @@ package org.apache.druid.data.input.thrift;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JSONParseSpec;
@ -29,6 +28,7 @@ import org.apache.druid.data.input.impl.JavaScriptParseSpec;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
@ -109,7 +109,7 @@ public class ThriftInputRowParserTest
// 2. binary + base64
serializer = new TSerializer(new TBinaryProtocol.Factory());
serializationAndTest(parser, Base64.encodeBase64(serializer.serialize(book)));
serializationAndTest(parser, StringUtils.encodeBase64(serializer.serialize(book)));
// 3. json
serializer = new TSerializer(new TJSONProtocol.Factory());

View File

@ -21,9 +21,9 @@ package org.apache.druid.query.aggregation.datasketches.hll;
import com.yahoo.memory.Memory;
import com.yahoo.sketches.hll.HllSketch;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.data.GenericIndexed;
@ -89,7 +89,7 @@ public class HllSketchMergeComplexMetricSerde extends ComplexMetricSerde
static HllSketch deserializeSketch(final Object object)
{
if (object instanceof String) {
return HllSketch.wrap(Memory.wrap(Base64.decodeBase64(((String) object).getBytes(StandardCharsets.UTF_8))));
return HllSketch.wrap(Memory.wrap(StringUtils.decodeBase64(((String) object).getBytes(StandardCharsets.UTF_8))));
} else if (object instanceof byte[]) {
return HllSketch.wrap(Memory.wrap((byte[]) object));
} else if (object instanceof HllSketch) {

View File

@ -21,8 +21,8 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
import com.yahoo.memory.Memory;
import com.yahoo.sketches.quantiles.DoublesSketch;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import java.nio.charset.StandardCharsets;
@ -47,7 +47,7 @@ public class DoublesSketchOperations
public static DoublesSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(Base64.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
}
public static DoublesSketch deserializeFromByteArray(final byte[] data)

View File

@ -31,7 +31,6 @@ import com.yahoo.sketches.theta.SetOperation;
import com.yahoo.sketches.theta.Sketch;
import com.yahoo.sketches.theta.Sketches;
import com.yahoo.sketches.theta.Union;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@ -223,7 +222,7 @@ public class SketchHolder
private static Sketch deserializeFromBase64EncodedString(String str)
{
return deserializeFromByteArray(Base64.decodeBase64(StringUtils.toUtf8(str)));
return deserializeFromByteArray(StringUtils.decodeBase64(StringUtils.toUtf8(str)));
}
private static Sketch deserializeFromByteArray(byte[] data)

View File

@ -27,9 +27,9 @@ import com.yahoo.sketches.tuple.ArrayOfDoublesSetOperationBuilder;
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
import com.yahoo.sketches.tuple.ArrayOfDoublesSketches;
import com.yahoo.sketches.tuple.ArrayOfDoublesUnion;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import java.nio.charset.StandardCharsets;
@ -117,7 +117,7 @@ public class ArrayOfDoublesSketchOperations
public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
{
return deserializeFromByteArray(Base64.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
}
public static ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data)

View File

@ -20,7 +20,7 @@
package org.apache.druid.query.aggregation.datasketches.hll;
import com.yahoo.sketches.hll.HllSketch;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import java.io.BufferedWriter;
import java.nio.charset.StandardCharsets;
@ -80,7 +80,7 @@ class GenerateTestData
out.write("\t");
out.write(dimension);
out.write("\t");
out.write(Base64.encodeBase64String(sketch.toCompactByteArray()));
out.write(StringUtils.encodeBase64String(sketch.toCompactByteArray()));
out.newLine();
}

View File

@ -21,7 +21,7 @@ package org.apache.druid.query.aggregation.datasketches.quantiles;
import com.yahoo.sketches.quantiles.DoublesSketch;
import com.yahoo.sketches.quantiles.UpdateDoublesSketch;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import java.io.BufferedWriter;
import java.nio.charset.StandardCharsets;
@ -63,7 +63,7 @@ public class GenerateTestData
sketchData.write('\t');
sketchData.write(Integer.toString(product)); // product dimension
sketchData.write('\t');
sketchData.write(Base64.encodeBase64String(sketch.toByteArray(true)));
sketchData.write(StringUtils.encodeBase64String(sketch.toByteArray(true)));
sketchData.newLine();
}
buildData.close();

View File

@ -21,7 +21,7 @@ package org.apache.druid.query.aggregation.datasketches.tuple;
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketch;
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import java.io.BufferedWriter;
import java.nio.charset.StandardCharsets;
@ -56,7 +56,7 @@ class GenerateTestData
out.write('\t');
out.write("product_" + (rand.nextInt(10) + 1));
out.write('\t');
out.write(Base64.encodeBase64String(sketch.compact().toByteArray()));
out.write(StringUtils.encodeBase64String(sketch.compact().toByteArray()));
out.newLine();
}
}

View File

@ -39,7 +39,6 @@ import java.io.IOException;
import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.security.spec.InvalidKeySpecException;
import java.util.Base64;
import java.util.HashMap;
import java.util.Map;
@ -136,7 +135,7 @@ public class BasicAuthUtils
public static String decodeUserSecret(String encodedUserSecret)
{
try {
return StringUtils.fromUtf8(Base64.getDecoder().decode(encodedUserSecret));
return StringUtils.fromUtf8(StringUtils.decodeBase64String(encodedUserSecret));
}
catch (IllegalArgumentException iae) {
return null;

View File

@ -19,7 +19,6 @@
package org.apache.druid.query.filter.sql;
import com.google.common.io.BaseEncoding;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
@ -28,6 +27,7 @@ import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.druid.guice.BloomFilterSerializersModule;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.filter.BloomDimFilter;
import org.apache.druid.query.filter.BloomKFilter;
import org.apache.druid.query.filter.BloomKFilterHolder;
@ -76,7 +76,7 @@ public class BloomFilterOperatorConversion implements SqlOperatorConversion
}
String base64EncodedBloomKFilter = RexLiteral.stringValue(operands.get(1));
final byte[] decoded = BaseEncoding.base64().decode(base64EncodedBloomKFilter);
final byte[] decoded = StringUtils.decodeBase64String(base64EncodedBloomKFilter);
BloomKFilter filter;
BloomKFilterHolder holder;
try {

View File

@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Key;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.guice.BloomFilterExtensionModule;
import org.apache.druid.guice.BloomFilterSerializersModule;
import org.apache.druid.guice.annotations.Json;
@ -90,7 +89,7 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
BloomKFilter filter = new BloomKFilter(1500);
filter.addString("def");
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
String base64 = Base64.encodeBase64String(bytes);
String base64 = StringUtils.encodeBase64String(bytes);
testQuery(
StringUtils.format("SELECT COUNT(*) FROM druid.foo WHERE bloom_filter_test(dim1, '%s')", base64),
@ -121,8 +120,8 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
filter.addString("abc");
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
byte[] bytes2 = BloomFilterSerializersModule.bloomKFilterToBytes(filter2);
String base64 = Base64.encodeBase64String(bytes);
String base642 = Base64.encodeBase64String(bytes2);
String base64 = StringUtils.encodeBase64String(bytes);
String base642 = StringUtils.encodeBase64String(bytes2);
testQuery(

View File

@ -19,7 +19,7 @@
package org.apache.druid.security.kerberos;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.security.authentication.server.AuthenticationToken;
@ -154,8 +154,7 @@ public class DruidKerberosAuthenticationHandler extends KerberosAuthenticationHa
authorization = authorization
.substring(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE.length())
.trim();
final Base64 base64 = new Base64(0);
final byte[] clientToken = base64.decode(authorization);
final byte[] clientToken = StringUtils.decodeBase64String(authorization);
final String serverName = request.getServerName();
try {
token = Subject.doAs(serverSubject, new PrivilegedExceptionAction<AuthenticationToken>()
@ -183,7 +182,7 @@ public class DruidKerberosAuthenticationHandler extends KerberosAuthenticationHa
gssContext = gssManager.createContext(gssCreds);
byte[] serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length);
if (serverToken != null && serverToken.length > 0) {
String authenticate = base64.encodeToString(serverToken);
String authenticate = StringUtils.encodeBase64String(serverToken);
response.setHeader(
org.apache.hadoop.security.authentication.client.KerberosAuthenticator.WWW_AUTHENTICATE,
org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE

View File

@ -20,8 +20,8 @@
package org.apache.druid.security.kerberos;
import com.google.common.base.Strings;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@ -47,8 +47,6 @@ public class DruidKerberosUtil
{
private static final Logger log = new Logger(DruidKerberosUtil.class);
private static final Base64 base64codec = new Base64(0);
// A fair reentrant lock
private static ReentrantLock kerberosLock = new ReentrantLock(true);
@ -81,7 +79,7 @@ public class DruidKerberosUtil
byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length);
gssContext.dispose();
// Base64 encoded and stringified token for server
return new String(base64codec.encode(outToken), StandardCharsets.US_ASCII);
return new String(StringUtils.encodeBase64(outToken), StandardCharsets.US_ASCII);
}
catch (GSSException | IllegalAccessException | NoSuchFieldException | ClassNotFoundException e) {
throw new AuthenticationException(e);

View File

@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.guice.annotations.Self;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
@ -561,8 +560,7 @@ public class KerberosAuthenticator implements Authenticator
} else {
authorization = authorization.substring(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE
.length()).trim();
final Base64 base64 = new Base64(0);
final byte[] clientToken = base64.decode(authorization);
final byte[] clientToken = StringUtils.decodeBase64String(authorization);
try {
DerInputStream ticketStream = new DerInputStream(clientToken);
DerValue[] values = ticketStream.getSet(clientToken.length, true);

View File

@ -24,7 +24,6 @@ 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 org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
@ -204,7 +203,7 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
return ah;
} else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(StringUtils.toUtf8((String) object));
byte[] bytes = StringUtils.decodeBase64(StringUtils.toUtf8((String) object));
final ApproximateHistogram ah = ApproximateHistogram.fromBytes(bytes);
ah.setLowerLimit(lowerLimit);
ah.setUpperLimit(upperLimit);

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@ -751,7 +750,7 @@ public class FixedBucketsHistogram
public String toBase64()
{
byte[] asBytes = toBytes();
return StringUtils.fromUtf8(Base64.encodeBase64(asBytes));
return StringUtils.fromUtf8(StringUtils.encodeBase64(asBytes));
}
/**
@ -902,7 +901,7 @@ public class FixedBucketsHistogram
*/
public static FixedBucketsHistogram fromBase64(String encodedHistogram)
{
byte[] asBytes = Base64.decodeBase64(encodedHistogram.getBytes(StandardCharsets.UTF_8));
byte[] asBytes = StringUtils.decodeBase64(encodedHistogram.getBytes(StandardCharsets.UTF_8));
return fromBytes(asBytes);
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.query.aggregation.histogram;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
@ -205,7 +204,7 @@ public class FixedBucketsHistogramAggregatorFactory extends AggregatorFactory
public Object deserialize(Object object)
{
if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(StringUtils.toUtf8((String) object));
byte[] bytes = StringUtils.decodeBase64(StringUtils.toUtf8((String) object));
final FixedBucketsHistogram fbh = FixedBucketsHistogram.fromBytes(bytes);
return fbh;
} else {

View File

@ -23,7 +23,6 @@ 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 org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregateCombiner;
@ -218,7 +217,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
return VarianceAggregatorCollector.from((ByteBuffer) object);
} else if (object instanceof String) {
return VarianceAggregatorCollector.from(
ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)))
ByteBuffer.wrap(StringUtils.decodeBase64(StringUtils.toUtf8((String) object)))
);
}
return object;

View File

@ -23,7 +23,6 @@ import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.junit.Assert;
@ -762,7 +761,7 @@ public class HyperLogLogCollectorTest
List<HyperLogLogCollector> collectors = Lists.transform(
objects,
s -> HyperLogLogCollector.makeCollector(ByteBuffer.wrap(Base64.decodeBase64(s)))
s -> HyperLogLogCollector.makeCollector(ByteBuffer.wrap(StringUtils.decodeBase64String(s)))
);
Collection<List<HyperLogLogCollector>> permutations = Collections2.permutations(collectors);

View File

@ -24,8 +24,8 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.StringUtils;
/**
*/
@ -78,7 +78,7 @@ public class GalaxyEC2UserData implements EC2UserData<GalaxyEC2UserData>
public String getUserDataBase64()
{
try {
return Base64.encodeBase64String(jsonMapper.writeValueAsBytes(this));
return StringUtils.encodeBase64String(jsonMapper.writeValueAsBytes(this));
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -21,7 +21,6 @@ package org.apache.druid.indexing.overlord.autoscaling.ec2;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import javax.annotation.Nullable;
@ -80,7 +79,7 @@ public class StringEC2UserData implements EC2UserData<StringEC2UserData>
} else {
finalData = data;
}
return Base64.encodeBase64String(StringUtils.toUtf8(finalData));
return StringUtils.encodeBase64String(StringUtils.toUtf8(finalData));
}
@Override

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.io.BaseEncoding;
import org.apache.druid.indexing.overlord.autoscaling.ec2.EC2AutoScaler;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
@ -128,8 +127,8 @@ public class EC2AutoScalerSerdeTest
Assert.assertEquals(
"VERSION=1234\n",
StringUtils.fromUtf8(
BaseEncoding.base64()
.decode(autoScaler.getEnvConfig().getUserData().withVersion("1234").getUserDataBase64())
StringUtils
.decodeBase64String(autoScaler.getEnvConfig().getUserData().withVersion("1234").getUserDataBase64())
)
);
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -149,7 +148,7 @@ public class HistogramAggregatorFactory extends AggregatorFactory
} else if (object instanceof ByteBuffer) {
return Histogram.fromBytes((ByteBuffer) object);
} else if (object instanceof String) {
byte[] bytes = Base64.decodeBase64(StringUtils.toUtf8((String) object));
byte[] bytes = StringUtils.decodeBase64(StringUtils.toUtf8((String) object));
return Histogram.fromBytes(bytes);
}
return object;

View File

@ -25,7 +25,6 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.ColumnSelectorPlus;
@ -227,7 +226,7 @@ public class CardinalityAggregatorFactory extends AggregatorFactory
// Be conservative, don't assume we own this buffer.
buffer = ((ByteBuffer) object).duplicate();
} else if (object instanceof String) {
buffer = ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)));
buffer = ByteBuffer.wrap(StringUtils.decodeBase64(StringUtils.toUtf8((String) object)));
} else {
return object;
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.hyperloglog;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
@ -185,7 +184,7 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
// Be conservative, don't assume we own this buffer.
buffer = ((ByteBuffer) object).duplicate();
} else if (object instanceof String) {
buffer = ByteBuffer.wrap(Base64.decodeBase64(StringUtils.toUtf8((String) object)));
buffer = ByteBuffer.wrap(StringUtils.decodeBase64(StringUtils.toUtf8((String) object)));
} else {
return object;
}

View File

@ -19,11 +19,11 @@
package org.apache.druid.query.aggregation.hyperloglog;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.hll.HyperLogLogHash;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import java.nio.ByteBuffer;
@ -59,7 +59,7 @@ public class PreComputedHyperUniquesSerde extends HyperUniquesSerde
return HyperLogLogCollector.makeLatestCollector().fold(ByteBuffer.wrap((byte[]) rawValue));
} else if (rawValue instanceof String) {
return HyperLogLogCollector.makeLatestCollector()
.fold(ByteBuffer.wrap(Base64.decodeBase64((String) rawValue)));
.fold(ByteBuffer.wrap(StringUtils.decodeBase64String((String) rawValue)));
}
throw new ISE("Object is not of a type[%s] that can be deserialized to HyperLogLog.", rawValue.getClass());

View File

@ -31,7 +31,6 @@ import com.google.common.collect.RangeSet;
import com.google.common.hash.Hasher;
import com.google.common.hash.Hashing;
import com.google.inject.Inject;
import org.apache.commons.codec.binary.Base64;
import org.apache.druid.client.cache.Cache;
import org.apache.druid.client.cache.CacheConfig;
import org.apache.druid.client.cache.CachePopulator;
@ -391,7 +390,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
if (hasOnlyHistoricalSegments) {
hasher.putBytes(queryCacheKey == null ? strategy.computeCacheKey(query) : queryCacheKey);
String currEtag = Base64.encodeBase64String(hasher.hash().asBytes());
String currEtag = StringUtils.encodeBase64String(hasher.hash().asBytes());
responseContext.put(QueryResource.HEADER_ETAG, currEtag);
return currEtag;
} else {