mirror of https://github.com/apache/druid.git
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:
parent
8492d94f59
commit
72a571fbf7
|
@ -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
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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())
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue