Prohibit String.replace() and String.replaceAll(), fix and prohibit some toString()-related redundancies (#6607)

* Prohibit String.replace() and String.replaceAll(), fix and prohibit some toString()-related redundancies

* Fix bug

* Replace checkstyle regexp with IntelliJ inspection
This commit is contained in:
Roman Leventov 2018-11-15 22:21:34 +01:00 committed by Slim Bouguerra
parent 0395d554e1
commit 8f3fe9cd02
101 changed files with 400 additions and 338 deletions

View File

@ -62,6 +62,7 @@
<inspection_tool class="ForLoopThatDoesntUseLoopVariable" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="HashCodeUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ImplicitArrayToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IndexOfReplaceableByContains" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InfiniteRecursion" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="InjectedReferences" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="InnerClassReferencedViaSubclass" enabled="true" level="ERROR" enabled_by_default="true" />
@ -71,6 +72,8 @@
<inspection_tool class="IteratorHasNextCallsIteratorNext" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="IteratorNextDoesNotThrowNoSuchElementException" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="JsonStandardCompliance" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="LengthOneStringInIndexOf" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ListIndexOfReplaceableByContains" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="MalformedFormatString" enabled="true" level="ERROR" enabled_by_default="true">
<option name="additionalClasses" value="org.apache.druid.java.util.common.StringUtils,org.apache.druid.java.util.common.logger.Logger" />
<option name="additionalMethods" value="trace,debug,info,warn,error,wtf,format,nonStrictFormat" />
@ -112,10 +115,13 @@
<inspection_tool class="ObjectEqualsNull" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ObjectToString" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="OverwrittenKey" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="PointlessIndexOfComparison" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="PrimitiveArrayArgumentToVariableArgMethod" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="RedundantStringOperation" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="RedundantThrows" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="RedundantTypeArguments" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ReflectionForUnavailableAnnotation" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="RemoveToStringInStringTemplate" enabled="true" level="WARNING" enabled_by_default="true" />
<inspection_tool class="ReplaceAllDot" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="ResultOfObjectAllocationIgnored" enabled="true" level="WARNING" enabled_by_default="true">
<scope name="Production" level="ERROR" enabled="true" />
@ -251,6 +257,7 @@
<inspection_tool class="ToArrayCallWithZeroLengthArrayArgument" enabled="true" level="WARNING" enabled_by_default="true">
<option name="myMode" value="BY_LEVEL" />
</inspection_tool>
<inspection_tool class="UnnecessaryCallToStringValueOf" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="UnnecessaryEnumModifier" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="UnnecessaryFullyQualifiedName" enabled="true" level="WARNING" enabled_by_default="true">
<scope name="NonGeneratedFiles" level="ERROR" enabled="true">
@ -261,6 +268,7 @@
<option name="ignoreInModuleStatements" value="true" />
</inspection_tool>
<inspection_tool class="UnnecessaryInterfaceModifier" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="UnnecessaryToStringCall" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="UnusedAssignment" enabled="true" level="ERROR" enabled_by_default="true">
<option name="REPORT_PREFIX_EXPRESSIONS" value="true" />
<option name="REPORT_POSTFIX_EXPRESSIONS" value="true" />

View File

@ -79,7 +79,7 @@ public class DimensionPredicateFilterBenchmark
if (input == null) {
return false;
}
return Integer.parseInt(input.toString()) % 2 == 0;
return Integer.parseInt(input) % 2 == 0;
}
};
}

View File

@ -139,7 +139,7 @@ public class FloatCompressionBenchmarkFileGenerator
// create compressed files using all combinations of CompressionStrategy and FloatEncoding provided
for (Map.Entry<String, BenchmarkColumnValueGenerator> entry : generators.entrySet()) {
for (CompressionStrategy compression : compressions) {
String name = entry.getKey() + "-" + compression.toString();
String name = entry.getKey() + "-" + compression;
log.info("%s: ", name);
File compFile = new File(dir, name);
compFile.delete();

View File

@ -132,7 +132,7 @@ public class LongCompressionBenchmarkFileGenerator
for (Map.Entry<String, BenchmarkColumnValueGenerator> entry : generators.entrySet()) {
for (CompressionStrategy compression : compressions) {
for (CompressionFactory.LongEncodingStrategy encoding : encodings) {
String name = entry.getKey() + "-" + compression.toString() + "-" + encoding.toString();
String name = entry.getKey() + "-" + compression + "-" + encoding;
log.info("%s: ", name);
File compFile = new File(dir, name);
compFile.delete();

View File

@ -443,7 +443,7 @@ public class BenchmarkDataGeneratorTest
Collections.sort(valList);
for (Comparable val : valList) {
System.out.println(" VAL: " + val.toString() + " CNT: " + valueMap.get(val));
System.out.println(" VAL: " + val + " CNT: " + valueMap.get(val));
}
System.out.println();
}

View File

@ -22,6 +22,11 @@ com.google.common.collect.Sets#newTreeSet() @ Create java.util.TreeSet directly
com.google.common.collect.Sets#newTreeSet(java.util.Comparator) @ Create java.util.TreeSet directly
com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use org.apache.druid.java.util.common.concurrent.ListenableFutures#transformAsync
java.io.File#toURL() @ Use java.io.File#toURI() and java.net.URI#toURL() instead
java.lang.String#matches(java.lang.String) @ Use startsWith(), endsWith(), contains(), or compile and cache a Pattern explicitly
java.lang.String#replace(java.lang.CharSequence,java.lang.CharSequence) @ Use one of the appropriate methods in StringUtils instead
java.lang.String#replaceAll(java.lang.String,java.lang.String) @ Use one of the appropriate methods in StringUtils instead, or compile and cache a Pattern explicitly
java.lang.String#replaceFirst(java.lang.String,java.lang.String) @ Use String.indexOf() and substring methods, or compile and cache a Pattern explicitly
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.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

View File

@ -153,16 +153,6 @@ public class StringUtils
return s.toUpperCase(Locale.ENGLISH);
}
public static String removeChar(String s, char c)
{
for (int i = 0; i < s.length(); i++) {
if (s.charAt(i) == c) {
return removeChar(s, c, i);
}
}
return s;
}
public static String urlEncode(String s)
{
try {
@ -173,16 +163,81 @@ public class StringUtils
}
}
private static String removeChar(String s, char c, int firstOccurranceIndex)
/**
* Removes all occurrences of the given char from the given string. This method is an optimal version of
* {@link String#replace(CharSequence, CharSequence) s.replace("c", "")}.
*/
public static String removeChar(String s, char c)
{
StringBuilder sb = new StringBuilder(s.length() - 1);
sb.append(s, 0, firstOccurranceIndex);
for (int i = firstOccurranceIndex + 1; i < s.length(); i++) {
char charOfString = s.charAt(i);
if (charOfString != c) {
sb.append(charOfString);
}
int pos = s.indexOf(c);
if (pos < 0) {
return s;
}
StringBuilder sb = new StringBuilder(s.length() - 1);
int prevPos = 0;
do {
sb.append(s, prevPos, pos);
prevPos = pos + 1;
pos = s.indexOf(c, pos + 1);
} while (pos > 0);
sb.append(s, prevPos, s.length());
return sb.toString();
}
/**
* Replaces all occurrences of the given char in the given string with the given replacement string. This method is an
* optimal version of {@link String#replace(CharSequence, CharSequence) s.replace("c", replacement)}.
*/
public static String replaceChar(String s, char c, String replacement)
{
int pos = s.indexOf(c);
if (pos < 0) {
return s;
}
StringBuilder sb = new StringBuilder(s.length() - 1 + replacement.length());
int prevPos = 0;
do {
sb.append(s, prevPos, pos);
sb.append(replacement);
prevPos = pos + 1;
pos = s.indexOf(c, pos + 1);
} while (pos > 0);
sb.append(s, prevPos, s.length());
return sb.toString();
}
/**
* Replaces all occurrences of the given target substring in the given string with the given replacement string. This
* method is an optimal version of {@link String#replace(CharSequence, CharSequence) s.replace(target, replacement)}.
*/
public static String replace(String s, String target, String replacement)
{
// String.replace() is suboptimal in JDK8, but is fixed in JDK9+. When the minimal JDK version supported by Druid is
// JDK9+, the implementation of this method should be replaced with simple delegation to String.replace(). However,
// the method should still be prohibited to use in all other places except this method body, because it's easy to
// suboptimally call String.replace("a", "b"), String.replace("a", ""), String.replace("a", "abc"), which have
// better alternatives String.replace('a', 'b'), removeChar() and replaceChar() respectively.
int pos = s.indexOf(target);
if (pos < 0) {
return s;
}
int sLength = s.length();
int targetLength = target.length();
// This is needed to work correctly with empty target string and mimic String.replace() behavior
int searchSkip = Math.max(targetLength, 1);
StringBuilder sb = new StringBuilder(sLength - targetLength + replacement.length());
int prevPos = 0;
do {
sb.append(s, prevPos, pos);
sb.append(replacement);
prevPos = pos + targetLength;
// Break from the loop if the target is empty
if (pos == sLength) {
break;
}
pos = s.indexOf(target, pos + searchSkip);
} while (pos > 0);
sb.append(s, prevPos, sLength);
return sb.toString();
}

View File

@ -139,8 +139,7 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
@Override
public byte[] getCacheKey()
{
return StringUtils.toUtf8(getPeriod().toString() + ":" +
getTimeZone().toString() + ":" + getOrigin());
return StringUtils.toUtf8(getPeriod() + ":" + getTimeZone() + ":" + getOrigin());
}
@Override
@ -420,8 +419,7 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
return t - offset;
} else {
throw new UnsupportedOperationException(
"Period cannot be converted to milliseconds as some fields mays vary in length with chronology "
+ chronology.toString()
"Period cannot be converted to milliseconds as some fields mays vary in length with chronology " + chronology
);
}
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.java.util.emitter.core;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
@ -48,7 +49,7 @@ public class ParametrizedUriEmitter implements Flushable, Closeable, Emitter
final ParametrizedUriExtractor parametrizedUriExtractor = new ParametrizedUriExtractor(baseUri);
UriExtractor uriExtractor = parametrizedUriExtractor;
if (ONLY_FEED_PARAM.equals(parametrizedUriExtractor.getParams())) {
uriExtractor = new FeedUriExtractor(baseUri.replace("{feed}", "%s"));
uriExtractor = new FeedUriExtractor(StringUtils.replace(baseUri, "{feed}", "%s"));
}
return uriExtractor;
}

View File

@ -65,7 +65,7 @@ public class ParametrizedUriExtractor implements UriExtractor
eventMap
);
}
processedUri = processedUri.replace(StringUtils.format("{%s}", key), paramValue.toString());
processedUri = StringUtils.replace(processedUri, StringUtils.format("{%s}", key), paramValue.toString());
}
return new URI(processedUri);
}

View File

@ -294,7 +294,7 @@ class UnaryMinusExpr extends UnaryExpr
@Override
public String toString()
{
return "-" + expr.toString();
return "-" + expr;
}
}
@ -321,7 +321,7 @@ class UnaryNotExpr extends UnaryExpr
@Override
public String toString()
{
return "!" + expr.toString();
return "!" + expr;
}
}

View File

@ -1033,14 +1033,12 @@ interface Function
}
final String arg = args.get(0).eval(bindings).asString();
final String pattern = args.get(1).eval(bindings).asString();
final String replacement = args.get(2).eval(bindings).asString();
final String pattern = NullHandling.nullToEmptyIfNeeded(args.get(1).eval(bindings).asString());
final String replacement = NullHandling.nullToEmptyIfNeeded(args.get(2).eval(bindings).asString());
if (arg == null) {
return ExprEval.of(NullHandling.defaultStringValue());
}
return ExprEval.of(
arg.replace(NullHandling.nullToEmptyIfNeeded(pattern), NullHandling.nullToEmptyIfNeeded(replacement))
);
return ExprEval.of(StringUtils.replace(arg, pattern, replacement));
}
}

View File

@ -24,6 +24,7 @@ import com.google.common.base.Joiner;
import com.google.common.base.Strings;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import org.apache.druid.java.util.common.StringUtils;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -102,7 +103,7 @@ public class UUIDUtilsTest
strings.add(uuidString.substring(16, 20));
strings.add(uuidString.substring(20, 32));
UUID uuid = UUID.fromString(Joiner.on('-').join(strings));
Assert.assertEquals(uuid.toString().replace("-", ""), uuidString);
Assert.assertEquals(StringUtils.removeChar(uuid.toString(), '-'), uuidString);
}
@Test

View File

@ -118,11 +118,34 @@ public class StringUtilsTest
}
@Test
public void testRemoveCharacter()
public void testRemoveChar()
{
Assert.assertEquals("123", StringUtils.removeChar("123", ','));
Assert.assertEquals("123", StringUtils.removeChar("123,", ','));
Assert.assertEquals("123", StringUtils.removeChar(",1,,2,3,", ','));
Assert.assertEquals("", StringUtils.removeChar(",,", ','));
}
@Test
public void testReplaceChar()
{
Assert.assertEquals("123", StringUtils.replaceChar("123", ',', "x"));
Assert.assertEquals("12345", StringUtils.replaceChar("123,", ',', "45"));
Assert.assertEquals("", StringUtils.replaceChar("", 'a', "bb"));
Assert.assertEquals("bb", StringUtils.replaceChar("a", 'a', "bb"));
Assert.assertEquals("bbbb", StringUtils.replaceChar("aa", 'a', "bb"));
}
@Test
public void testReplace()
{
Assert.assertEquals("x1x2x3x", StringUtils.replace("123", "", "x"));
Assert.assertEquals("12345", StringUtils.replace("123,", ",", "45"));
Assert.assertEquals("", StringUtils.replace("", "a", "bb"));
Assert.assertEquals("bb", StringUtils.replace("a", "a", "bb"));
Assert.assertEquals("bba", StringUtils.replace("aaa", "aa", "bb"));
Assert.assertEquals("bcb", StringUtils.replace("aacaa", "aa", "b"));
Assert.assertEquals("bb", StringUtils.replace("aaaa", "aa", "b"));
Assert.assertEquals("", StringUtils.replace("aaaa", "aa", ""));
}
}

View File

@ -137,7 +137,7 @@ public class ParametrizedUriEmitterTest
protected ListenableFuture<Response> go(Request request)
{
results.put(
request.getUrl().toString(),
request.getUrl(),
StandardCharsets.UTF_8.decode(request.getByteBufferData().slice()).toString()
);
return GoHandlers.immediateFuture(okResponse());

View File

@ -238,7 +238,7 @@ public class FriendlyServersTest
}
Assert.assertTrue("ChannelException thrown by 'get'", ea instanceof ChannelException);
Assert.assertTrue("Expected error message", ea.getCause().getMessage().matches(".*Failed to handshake.*"));
Assert.assertTrue("Expected error message", ea.getCause().getMessage().contains("Failed to handshake"));
}
{

View File

@ -298,7 +298,7 @@ public class JankyServersTest
public boolean isChannelClosedException(Throwable e)
{
return e instanceof ChannelException ||
(e instanceof IOException && e.getMessage().matches(".*Connection reset by peer.*"));
(e instanceof IOException && e.getMessage().contains("Connection reset by peer"));
}
@Test

View File

@ -25,7 +25,6 @@ import org.junit.Assert;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.regex.Pattern;
public class TestUtils
{
@ -41,10 +40,7 @@ public class TestUtils
final String procMountsString = StringUtils.fromUtf8(Files.readAllBytes(procMountsTemplate.toPath()));
Files.write(
procMounts.toPath(),
StringUtils.toUtf8(procMountsString.replaceAll(
Pattern.quote("/sys/fs/cgroup"),
cgroupDir.getAbsolutePath()
))
StringUtils.toUtf8(StringUtils.replace(procMountsString, "/sys/fs/cgroup", cgroupDir.getAbsolutePath()))
);
Assert.assertTrue(new File(

View File

@ -46,17 +46,16 @@ import java.util.zip.GZIPInputStream;
class WikipediaIrcDecoder implements IrcDecoder
{
static final Logger log = new Logger(WikipediaIrcDecoder.class);
static final Logger LOG = new Logger(WikipediaIrcDecoder.class);
final DatabaseReader geoLookup;
static final Pattern pattern = Pattern.compile(
private static final Pattern PATTERN = Pattern.compile(
".*\\x0314\\[\\[\\x0307(.+?)\\x0314\\]\\]\\x034 (.*?)\\x0310.*\\x0302(http.+?)" +
"\\x03.+\\x0303(.+?)\\x03.+\\x03 (\\(([+-]\\d+)\\).*|.+) \\x0310(.+)\\x03.*"
);
static final Pattern ipPattern = Pattern.compile("\\d+.\\d+.\\d+.\\d+");
static final Pattern shortnamePattern = Pattern.compile("#(\\w\\w)\\..*");
private static final Pattern IP_PATTERN = Pattern.compile("\\d+.\\d+.\\d+.\\d+");
private static final Pattern SHORTNAME_PATTERN = Pattern.compile("#(\\w\\w)\\..*");
private static final Pattern SINGLE_SPACE_PATTERN = Pattern.compile("\\s");
static final List<String> dimensionList = Lists.newArrayList(
"page",
@ -73,6 +72,7 @@ class WikipediaIrcDecoder implements IrcDecoder
"city"
);
final DatabaseReader geoLookup;
final Map<String, Map<String, String>> namespaces;
final String geoIpDatabase;
@ -108,7 +108,7 @@ class WikipediaIrcDecoder implements IrcDecoder
return openDefaultGeoIpDb(geoDb);
}
catch (RuntimeException e) {
log.warn(e.getMessage() + " Attempting to re-download.", e);
LOG.warn(e.getMessage() + " Attempting to re-download.", e);
if (geoDb.exists() && !geoDb.delete()) {
throw new RuntimeException("Could not delete geo db file [" + geoDb.getAbsolutePath() + "].");
}
@ -127,7 +127,7 @@ class WikipediaIrcDecoder implements IrcDecoder
{
try {
DatabaseReader reader = new DatabaseReader(geoDb);
log.info("Using geo ip database at [%s].", geoDb);
LOG.info("Using geo ip database at [%s].", geoDb);
return reader;
}
catch (IOException e) {
@ -142,7 +142,7 @@ class WikipediaIrcDecoder implements IrcDecoder
}
try {
log.info("Downloading geo ip database to [%s]. This may take a few minutes.", geoDb.getAbsolutePath());
LOG.info("Downloading geo ip database to [%s]. This may take a few minutes.", geoDb.getAbsolutePath());
File tmpFile = File.createTempFile("druid", "geo");
@ -180,23 +180,23 @@ class WikipediaIrcDecoder implements IrcDecoder
final Map<String, String> dimensions = new HashMap<>();
final Map<String, Float> metrics = new HashMap<>();
Matcher m = pattern.matcher(msg);
Matcher m = PATTERN.matcher(msg);
if (!m.matches()) {
throw new IllegalArgumentException("Invalid input format");
}
Matcher shortname = shortnamePattern.matcher(channel);
Matcher shortname = SHORTNAME_PATTERN.matcher(channel);
if (shortname.matches()) {
dimensions.put("language", shortname.group(1));
}
String page = m.group(1);
String pageUrl = page.replaceAll("\\s", "_");
String pageUrl = SINGLE_SPACE_PATTERN.matcher(page).replaceAll("_");
dimensions.put("page", pageUrl);
String user = m.group(4);
Matcher ipMatch = ipPattern.matcher(user);
Matcher ipMatch = IP_PATTERN.matcher(user);
boolean anonymous = ipMatch.matches();
if (anonymous) {
try {
@ -209,13 +209,13 @@ class WikipediaIrcDecoder implements IrcDecoder
dimensions.put("city", lookup.getCity().getName());
}
catch (UnknownHostException e) {
log.error(e, "invalid ip [%s]", ipMatch.group());
LOG.error(e, "invalid ip [%s]", ipMatch.group());
}
catch (IOException e) {
log.error(e, "error looking up geo ip");
LOG.error(e, "error looking up geo ip");
}
catch (GeoIp2Exception e) {
log.error(e, "error looking up geo ip");
LOG.error(e, "error looking up geo ip");
}
}
dimensions.put("user", user);

View File

@ -95,7 +95,7 @@ public class AzureDataSegmentPusher implements DataSegmentPusher
dataSegment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
dataSegment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
),
dataSegment.getVersion().replace(":", "_"),
dataSegment.getVersion().replace(':', '_'),
dataSegment.getShardSpec().getPartitionNum(),
useUniquePath ? DataSegmentPusher.generateUniquePath() : null
);

View File

@ -42,6 +42,7 @@ import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;
import static org.easymock.EasyMock.expectLastCall;
import static org.junit.Assert.assertEquals;
@ -119,7 +120,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
Assert.assertTrue(
segment.getLoadSpec().get("blobPath").toString(),
segment.getLoadSpec().get("blobPath").toString().matches(matcher)
Pattern.compile(matcher).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
);
Assert.assertEquals(segmentToPush.getSize(), segment.getSize());

View File

@ -58,7 +58,7 @@ public class CloudFilesUtils
{
String path = fileName;
if (!basePath.isEmpty()) {
int lastSlashIndex = basePath.lastIndexOf("/");
int lastSlashIndex = basePath.lastIndexOf('/');
if (lastSlashIndex != -1) {
basePath = basePath.substring(0, lastSlashIndex);
}

View File

@ -52,7 +52,7 @@ public class GoogleDataSegmentKiller implements DataSegmentKiller
Map<String, Object> loadSpec = segment.getLoadSpec();
final String bucket = MapUtils.getString(loadSpec, "bucket");
final String indexPath = MapUtils.getString(loadSpec, "path");
final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf("/")) + "/descriptor.json";
final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf('/')) + "/descriptor.json";
try {
deleteIfPresent(bucket, indexPath);

View File

@ -131,12 +131,12 @@ public class GoogleTaskLogs implements TaskLogs
private String getTaskLogKey(String taskid)
{
return config.getPrefix() + "/" + taskid.replaceAll(":", "_");
return config.getPrefix() + "/" + taskid.replace(':', '_');
}
private String getTaskReportKey(String taskid)
{
return config.getPrefix() + "/" + taskid.replaceAll(":", "_") + ".report.json";
return config.getPrefix() + "/" + taskid.replace(':', '_') + ".report.json";
}
@Override

View File

@ -27,14 +27,12 @@ public class GoogleUtils
{
public static String toFilename(String path)
{
String filename = path.substring(path.lastIndexOf("/") + 1); // characters after last '/'
filename = filename.substring(0, filename.length());
return filename;
return path.substring(path.lastIndexOf('/') + 1); // characters after last '/'
}
public static String indexZipForSegmentPath(String path)
{
return path.substring(0, path.lastIndexOf("/")) + "/index.zip";
return path.substring(0, path.lastIndexOf('/')) + "/index.zip";
}
public static boolean isRetryable(Throwable t)

View File

@ -40,7 +40,7 @@ public class GoogleDataSegmentKillerTest extends EasyMockSupport
{
private static final String bucket = "bucket";
private static final String indexPath = "test/2015-04-12T00:00:00.000Z_2015-04-13T00:00:00.000Z/1/0/index.zip";
private static final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf("/")) + "/descriptor.json";
private static final String descriptorPath = indexPath.substring(0, indexPath.lastIndexOf('/')) + "/descriptor.json";
private static final DataSegment dataSegment = new DataSegment(
"test",

View File

@ -252,7 +252,7 @@ public class GraphiteEmitter implements Emitter
Pattern DOT_OR_WHITESPACE = Pattern.compile("[\\s]+|[.]+");
String sanitizedNamespace = DOT_OR_WHITESPACE.matcher(namespace).replaceAll("_");
if (replaceSlashToDot) {
sanitizedNamespace = sanitizedNamespace.replace("/", ".");
sanitizedNamespace = sanitizedNamespace.replace('/', '.');
}
return sanitizedNamespace;
}

View File

@ -31,11 +31,16 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class InfluxParser implements Parser<String, Object>
{
public static final String TIMESTAMP_KEY = "__ts";
private static final String MEASUREMENT_KEY = "measurement";
private static final Pattern BACKSLASH_PATTERN = Pattern.compile("\\\\\"");
private static final Pattern IDENTIFIER_PATTERN = Pattern.compile("\\\\([,= ])");
private final Set<String> measurementWhitelist;
public InfluxParser(Set<String> measurementWhitelist)
@ -112,7 +117,7 @@ public class InfluxParser implements Parser<String, Object>
private Object parseQuotedString(String text)
{
return text.substring(1, text.length() - 1).replaceAll("\\\\\"", "\"");
return BACKSLASH_PATTERN.matcher(text.substring(1, text.length() - 1)).replaceAll("\"");
}
private Object parseNumber(String raw)
@ -140,7 +145,7 @@ public class InfluxParser implements Parser<String, Object>
return ctx.getText();
}
return ctx.IDENTIFIER_STRING().getText().replaceAll("\\\\([,= ])", "$1");
return IDENTIFIER_PATTERN.matcher(ctx.IDENTIFIER_STRING().getText()).replaceAll("$1");
}
private boolean checkWhitelist(String m)

View File

@ -208,8 +208,8 @@ public class MaterializedViewQuery<T> implements Query<T>
public String toString()
{
return "MaterializedViewQuery{" +
"query=" + query.toString() +
"}";
"query=" + query +
"}";
}
@Override

View File

@ -39,8 +39,6 @@ public class EventConverter
{
private static final Logger log = new Logger(EventConverter.class);
private static final Pattern WHITESPACE = Pattern.compile("[\\s]+");
private static final String COLON = ":";
private static final String DEFAULT_COLON_REPLACEMENT = "_";
private final Map<String, Set<String>> metricMap;
@ -51,7 +49,7 @@ public class EventConverter
protected String sanitize(String metric)
{
return WHITESPACE.matcher(metric.trim()).replaceAll("_").replaceAll("/", ".");
return WHITESPACE.matcher(metric.trim()).replaceAll("_").replace('/', '.');
}
/**
@ -74,8 +72,8 @@ public class EventConverter
Number value = serviceMetricEvent.getValue();
Map<String, Object> tags = new HashMap<>();
String service = serviceMetricEvent.getService().replaceAll(COLON, DEFAULT_COLON_REPLACEMENT);
String host = serviceMetricEvent.getHost().replaceAll(COLON, DEFAULT_COLON_REPLACEMENT);
String service = serviceMetricEvent.getService().replace(':', '_');
String host = serviceMetricEvent.getHost().replace(':', '_');
tags.put("service", service);
tags.put("host", host);
@ -84,7 +82,7 @@ public class EventConverter
if (userDims.containsKey(dim)) {
Object dimValue = userDims.get(dim);
if (dimValue instanceof String) {
dimValue = ((String) dimValue).replaceAll(COLON, DEFAULT_COLON_REPLACEMENT);
dimValue = ((String) dimValue).replace(':', '_');
}
tags.put(dim, dimValue);
}

View File

@ -84,9 +84,9 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
this.typeString = typeString == null ? typeStringFromParseSpec(parseSpec) : typeString;
this.mapFieldNameFormat =
mapFieldNameFormat == null ||
mapFieldNameFormat.indexOf(MAP_PARENT_TAG) < 0 ||
mapFieldNameFormat.indexOf(MAP_CHILD_TAG) < 0 ? DEFAULT_MAP_FIELD_NAME_FORMAT : mapFieldNameFormat;
this.mapParentFieldNameFormat = this.mapFieldNameFormat.replace(MAP_PARENT_TAG, "%s");
!mapFieldNameFormat.contains(MAP_PARENT_TAG) ||
!mapFieldNameFormat.contains(MAP_CHILD_TAG) ? DEFAULT_MAP_FIELD_NAME_FORMAT : mapFieldNameFormat;
this.mapParentFieldNameFormat = StringUtils.replace(this.mapFieldNameFormat, MAP_PARENT_TAG, "%s");
this.dimensions = parseSpec.getDimensionsSpec().getDimensionNames();
this.oip = makeObjectInspector(this.typeString);
}
@ -159,7 +159,11 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
if (mapObjectInspector.getMapSize(mapObject) < 0) {
return;
}
String mapChildFieldNameFormat = StringUtils.format(mapParentFieldNameFormat, parentName).replace(MAP_CHILD_TAG, "%s");
String mapChildFieldNameFormat = StringUtils.replace(
StringUtils.format(mapParentFieldNameFormat, parentName),
MAP_CHILD_TAG,
"%s"
);
Map objectMap = mapObjectInspector.getMap(mapObject);
PrimitiveObjectInspector key = (PrimitiveObjectInspector) mapObjectInspector.getMapKeyObjectInspector();

View File

@ -25,12 +25,14 @@ import com.google.common.collect.ImmutableList;
import com.timgroup.statsd.NonBlockingStatsDClient;
import com.timgroup.statsd.StatsDClient;
import com.timgroup.statsd.StatsDClientErrorHandler;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.core.Emitter;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import java.util.Map;
import java.util.regex.Pattern;
/**
*/
@ -38,9 +40,9 @@ public class StatsDEmitter implements Emitter
{
private static final Logger log = new Logger(StatsDEmitter.class);
private static final String DRUID_METRIC_SEPARATOR = "\\/";
private static final String STATSD_SEPARATOR = ":|\\|";
private static final String BLANK = "\\s+";
private static final char DRUID_METRIC_SEPARATOR = '/';
private static final Pattern STATSD_SEPARATOR = Pattern.compile("[:|]");
private static final Pattern BLANK = Pattern.compile("\\s+");
static final StatsDEmitter of(StatsDEmitterConfig config, ObjectMapper mapper)
{
@ -101,11 +103,10 @@ public class StatsDEmitter implements Emitter
if (statsDMetric != null) {
String fullName = Joiner.on(config.getSeparator())
.join(nameBuilder.build())
.replaceAll(DRUID_METRIC_SEPARATOR, config.getSeparator())
.replaceAll(STATSD_SEPARATOR, config.getSeparator())
.replaceAll(BLANK, config.getBlankHolder());
String fullName = Joiner.on(config.getSeparator()).join(nameBuilder.build());
fullName = StringUtils.replaceChar(fullName, DRUID_METRIC_SEPARATOR, config.getSeparator());
fullName = STATSD_SEPARATOR.matcher(fullName).replaceAll(config.getSeparator());
fullName = BLANK.matcher(fullName).replaceAll(config.getBlankHolder());
long val = statsDMetric.convertRange ? Math.round(value.doubleValue() * 100) : value.longValue();
switch (statsDMetric.type) {

View File

@ -64,6 +64,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import static org.junit.Assert.assertEquals;
@ -150,6 +151,8 @@ public class AvroStreamInputRowParserTest
public static final String SOME_UNION_VALUE = "string as union";
public static final ByteBuffer SOME_BYTES_VALUE = ByteBuffer.allocate(8);
private static final Pattern BRACES_AND_SPACE = Pattern.compile("[{} ]");
private final ObjectMapper jsonMapper = new ObjectMapper();
@ -267,7 +270,7 @@ public class AvroStreamInputRowParserTest
assertEquals(1543698L, inputRow.getTimestampFromEpoch());
// test dimensions
assertEquals(Collections.singletonList(String.valueOf(EVENT_TYPE_VALUE)), inputRow.getDimension(EVENT_TYPE));
assertEquals(Collections.singletonList(EVENT_TYPE_VALUE), inputRow.getDimension(EVENT_TYPE));
assertEquals(Collections.singletonList(String.valueOf(ID_VALUE)), inputRow.getDimension(ID));
assertEquals(Collections.singletonList(String.valueOf(SOME_OTHER_ID_VALUE)), inputRow.getDimension(SOME_OTHER_ID));
assertEquals(Collections.singletonList(String.valueOf(true)), inputRow.getDimension(IS_VALID));
@ -285,9 +288,10 @@ public class AvroStreamInputRowParserTest
SOME_INT_VALUE_MAP_VALUE,
new HashMap<CharSequence, Integer>(
Maps.transformValues(
Splitter.on(",")
.withKeyValueSeparator("=")
.split(inputRow.getDimension("someIntValueMap").get(0).replaceAll("[\\{\\} ]", "")),
Splitter
.on(",")
.withKeyValueSeparator("=")
.split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll("")),
new Function<String, Integer>()
{
@Nullable
@ -303,9 +307,10 @@ public class AvroStreamInputRowParserTest
assertEquals(
SOME_STRING_VALUE_MAP_VALUE,
new HashMap<CharSequence, CharSequence>(
Splitter.on(",")
.withKeyValueSeparator("=")
.split(inputRow.getDimension("someIntValueMap").get(0).replaceAll("[\\{\\} ]", ""))
Splitter
.on(",")
.withKeyValueSeparator("=")
.split(BRACES_AND_SPACE.matcher(inputRow.getDimension("someIntValueMap").get(0)).replaceAll(""))
)
);
assertEquals(Collections.singletonList(SOME_UNION_VALUE), inputRow.getDimension("someUnion"));

View File

@ -19,24 +19,22 @@
package org.apache.druid.query.aggregation.datasketches.hll;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
import javax.annotation.Nullable;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.yahoo.sketches.hll.HllSketch;
import com.yahoo.sketches.hll.TgtHllType;
import com.yahoo.sketches.hll.Union;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.ObjectAggregateCombiner;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
/**
* Base class for both build and merge factories
* @author Alexander Saydakov
@ -176,7 +174,7 @@ abstract class HllSketchAggregatorFactory extends AggregatorFactory
@Override
public AggregatorFactory getCombiningFactory()
{
return new HllSketchMergeAggregatorFactory(getName(), getName(), getLgK(), getTgtHllType().toString());
return new HllSketchMergeAggregatorFactory(getName(), getName(), getLgK(), getTgtHllType());
}
@Override

View File

@ -74,11 +74,11 @@ public class SketchEstimateWithErrorBounds
public String toString()
{
return "SketchEstimateWithErrorBounds{" +
"estimate=" + Double.toString(estimate) +
", highBound=" + Double.toString(highBound) +
", lowBound=" + Double.toString(lowBound) +
", numStdDev=" + Integer.toString(numStdDev) +
"}";
"estimate=" + estimate +
", highBound=" + highBound +
", lowBound=" + lowBound +
", numStdDev=" + numStdDev +
"}";
}
@Override

View File

@ -433,7 +433,7 @@ public class BloomKFilter
this.bitSet.putAll(that.bitSet);
} else {
throw new IllegalArgumentException("BloomKFilters are not compatible for merging." +
" this - " + this.toString() + " that - " + that.toString());
" this - " + this + " that - " + that);
}
}

View File

@ -67,7 +67,7 @@ public class HdfsDataSegmentFinder implements DataSegmentFinder
fs = workingDirPath.getFileSystem(config);
log.info(fs.getScheme());
log.info("FileSystem URI:" + fs.getUri().toString());
log.info("FileSystem URI:" + fs.getUri());
if (!fs.exists(workingDirPath)) {
throw new SegmentLoadingException("Working directory [%s] doesn't exist.", workingDirPath);

View File

@ -242,7 +242,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
),
segment.getVersion().replaceAll(":", "_")
segment.getVersion().replace(':', '_')
);
}

View File

@ -135,7 +135,7 @@ public class HdfsTaskLogs implements TaskLogs
*/
private Path getTaskLogFileFromId(String taskId)
{
return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_")));
return new Path(mergePaths(config.getDirectory(), taskId.replace(':', '_')));
}
/**
@ -144,7 +144,7 @@ public class HdfsTaskLogs implements TaskLogs
*/
private Path getTaskReportsFileFromId(String taskId)
{
return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_") + ".reports.json"));
return new Path(mergePaths(config.getDirectory(), taskId.replace(':', '_') + ".reports.json"));
}
// some hadoop version Path.mergePaths does not exist

View File

@ -306,13 +306,13 @@ public class HdfsDataSegmentFinderTest
private String getDescriptorPath(DataSegment segment)
{
final Path indexzip = new Path(String.valueOf(segment.getLoadSpec().get("path")));
return indexzip.getParent().toString() + "/" + DESCRIPTOR_JSON;
return indexzip.getParent() + "/" + DESCRIPTOR_JSON;
}
private String getDescriptorPathWithPartitionNum(DataSegment segment, int partitionNum)
{
final Path indexzip = new Path(String.valueOf(segment.getLoadSpec().get("path")));
return indexzip.getParent().toString() + "/" + partitionNum + "_" + DESCRIPTOR_JSON;
return indexzip.getParent() + "/" + partitionNum + "_" + DESCRIPTOR_JSON;
}
private String readContent(Path descriptor) throws IOException

View File

@ -118,7 +118,7 @@ public class HdfsDataSegmentPullerTest
final File outTmpDir = com.google.common.io.Files.createTempDir();
final URI uri = URI.create(uriBase.toString() + zipPath.toString());
final URI uri = URI.create(uriBase.toString() + zipPath);
try (final OutputStream stream = new FileOutputStream(tmpFile)) {
ByteStreams.copy(new ByteArrayInputStream(pathByteContents), stream);
@ -163,7 +163,7 @@ public class HdfsDataSegmentPullerTest
final File outFile = new File(outTmpDir, "testZip");
outFile.delete();
final URI uri = URI.create(uriBase.toString() + zipPath.toString());
final URI uri = URI.create(uriBase.toString() + zipPath);
try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath);
final OutputStream gzStream = new GZIPOutputStream(outputStream);
@ -197,7 +197,7 @@ public class HdfsDataSegmentPullerTest
final File outFile = new File(outTmpDir, "test.txt");
outFile.delete();
final URI uri = URI.create(uriBase.toString() + perTestPath.toString());
final URI uri = URI.create(uriBase.toString() + perTestPath);
try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath);
final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {

View File

@ -68,6 +68,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.regex.Pattern;
/**
*/
@ -161,10 +162,11 @@ public class HdfsDataSegmentPusherTest
DataSegment segment = pusher.push(segmentDir, segmentToPush, true);
String matcher = ".*/foo/20150101T000000\\.000Z_20160101T000000\\.000Z/0/0_[A-Za-z0-9-]{36}_index\\.zip";
Pattern pattern =
Pattern.compile(".*/foo/20150101T000000\\.000Z_20160101T000000\\.000Z/0/0_[A-Za-z0-9-]{36}_index\\.zip");
Assert.assertTrue(
segment.getLoadSpec().get("path").toString(),
segment.getLoadSpec().get("path").toString().matches(matcher)
pattern.matcher(segment.getLoadSpec().get("path").toString()).matches()
);
}

View File

@ -117,7 +117,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory
this.cacheManager = cacheManager;
this.connectTimeout = connectTimeout;
this.injective = injective;
this.factoryId = "kafka-factory-" + kafkaTopic + UUID.randomUUID().toString();
this.factoryId = "kafka-factory-" + kafkaTopic + UUID.randomUUID();
}
public KafkaLookupExtractorFactory(

View File

@ -512,8 +512,7 @@ public class UriCacheGeneratorTest
{
Assert.assertNull(scheduler.scheduleAndWait(
new UriExtractionNamespace(
new URI("file://tmp/I_DONT_REALLY_EXIST" +
UUID.randomUUID().toString()),
new URI("file://tmp/I_DONT_REALLY_EXIST" + UUID.randomUUID()),
null,
null,
new UriExtractionNamespace.JSONFlatDataParser(

View File

@ -192,7 +192,7 @@ public class OffHeapLoadingCache<K, V> implements LoadingCache<K, V>
public void close()
{
if (!closed.getAndSet(true)) {
DB.delete(String.valueOf(name));
DB.delete(name);
}
}

View File

@ -180,7 +180,7 @@ public class MySQLConnector extends SQLMetadataConnector
.map(StringMapper.FIRST)
.first();
if (!databaseCharset.matches("utf8.*")) {
if (!databaseCharset.startsWith("utf8")) {
throw new ISE(
"Druid requires its MySQL database to be created with an UTF8 charset, found `%1$s`. "
+ "The recommended charset is `utf8mb4`.",

View File

@ -122,7 +122,7 @@ public class ProtobufInputRowParser implements ByteBufferInputRowParser
fin = url.openConnection().getInputStream();
}
catch (IOException e) {
throw new ParseException(e, "Cannot read descriptor file: " + url.toString());
throw new ParseException(e, "Cannot read descriptor file: " + url);
}
}

View File

@ -288,9 +288,7 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
final String authority = originalAuthority.endsWith("/") ?
originalAuthority.substring(0, originalAuthority.length() - 1) :
originalAuthority;
final String path = originalPath.startsWith("/") ?
originalPath.substring(1, originalPath.length()) :
originalPath;
final String path = originalPath.startsWith("/") ? originalPath.substring(1) : originalPath;
return URI.create(StringUtils.format("s3://%s/%s", authority, path));
}

View File

@ -174,12 +174,12 @@ public class S3Utils
static String descriptorPathForSegmentPath(String s3Path)
{
return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json";
return s3Path.substring(0, s3Path.lastIndexOf('/')) + "/descriptor.json";
}
static String indexZipForSegmentPath(String s3Path)
{
return s3Path.substring(0, s3Path.lastIndexOf("/")) + "/index.zip";
return s3Path.substring(0, s3Path.lastIndexOf('/')) + "/index.zip";
}
static String toFilename(String key)
@ -189,7 +189,7 @@ public class S3Utils
static String toFilename(String key, final String suffix)
{
String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/'
String filename = key.substring(key.lastIndexOf('/') + 1); // characters after last '/'
filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end
return filename;
}

View File

@ -45,6 +45,7 @@ import java.io.File;
import java.io.FileInputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.regex.Pattern;
/**
*/
@ -148,7 +149,7 @@ public class S3DataSegmentPusherTest
Assert.assertEquals("bucket", segment.getLoadSpec().get("bucket"));
Assert.assertTrue(
segment.getLoadSpec().get("key").toString(),
segment.getLoadSpec().get("key").toString().matches(matcher)
Pattern.compile(matcher).matcher(segment.getLoadSpec().get("key").toString()).matches()
);
Assert.assertEquals("s3_zip", segment.getLoadSpec().get("type"));

View File

@ -507,7 +507,7 @@ public class HadoopDruidIndexerConfig
"%s/%s/%s_%s",
getWorkingPath(),
schema.getDataSchema().getDataSource(),
schema.getTuningConfig().getVersion().replace(":", ""),
StringUtils.removeChar(schema.getTuningConfig().getVersion(), ':'),
schema.getUniqueId()
)
);
@ -547,7 +547,10 @@ public class HadoopDruidIndexerConfig
public Path makeDescriptorInfoPath(DataSegment segment)
{
return new Path(makeDescriptorInfoDir(), StringUtils.format("%s.json", segment.getIdentifier().replace(":", "")));
return new Path(
makeDescriptorInfoDir(),
StringUtils.removeChar(StringUtils.format("%s.json", segment.getIdentifier()), ':')
);
}
public void addJobProperties(Job job)

View File

@ -766,7 +766,11 @@ public class JobHelper
// getHdfsStorageDir. But that wouldn't fix this issue for people who already have segments with ":".
// Because of this we just URL encode the : making everything work as it should.
segmentLocURI = URI.create(
StringUtils.format("gs://%s/%s", loadSpec.get("bucket"), loadSpec.get("path").toString().replace(":", "%3A"))
StringUtils.format(
"gs://%s/%s",
loadSpec.get("bucket"),
StringUtils.replaceChar(loadSpec.get("path").toString(), ':', "%3A")
)
);
} else if ("local".equals(type)) {
try {

View File

@ -58,7 +58,7 @@ public class Utils
if (FileOutputFormat.getCompressOutput(job)) {
codecClass = FileOutputFormat.getOutputCompressorClass(job, GzipCodec.class);
codec = ReflectionUtils.newInstance(codecClass, job.getConfiguration());
outputPath = new Path(outputPath.toString() + codec.getDefaultExtension());
outputPath = new Path(outputPath + codec.getDefaultExtension());
}
if (fs.exists(outputPath)) {
@ -89,7 +89,7 @@ public class Utils
} else {
Class<? extends CompressionCodec> codecClass = FileOutputFormat.getOutputCompressorClass(job, GzipCodec.class);
CompressionCodec codec = ReflectionUtils.newInstance(codecClass, job.getConfiguration());
return fs.exists(new Path(inputPath.toString() + codec.getDefaultExtension()));
return fs.exists(new Path(inputPath + codec.getDefaultExtension()));
}
}
@ -101,7 +101,7 @@ public class Utils
} else {
Class<? extends CompressionCodec> codecClass = FileOutputFormat.getOutputCompressorClass(job, GzipCodec.class);
CompressionCodec codec = ReflectionUtils.newInstance(codecClass, job.getConfiguration());
inputPath = new Path(inputPath.toString() + codec.getDefaultExtension());
inputPath = new Path(inputPath + codec.getDefaultExtension());
return codec.createInputStream(fileSystem.open(inputPath));
}

View File

@ -141,7 +141,7 @@ public class HadoopConverterJob
public static Path getJobClassPathDir(String jobName, Path workingDirectory)
{
return new Path(workingDirectory, jobName.replace(":", ""));
return new Path(workingDirectory, StringUtils.removeChar(jobName, ':'));
}
public static void cleanup(Job job) throws IOException

View File

@ -68,7 +68,7 @@ public class UtilsTest
@Override
public Object apply(Object input)
{
return input.toString() + DUMMY_STRING;
return input + DUMMY_STRING;
}
}

View File

@ -24,16 +24,20 @@ 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;
import java.util.Objects;
public class StringEC2UserData implements EC2UserData<StringEC2UserData>
{
private final String data;
@Nullable
private final String versionReplacementString;
private final String version;
@JsonCreator
public StringEC2UserData(
@JsonProperty("data") String data,
@JsonProperty("versionReplacementString") String versionReplacementString,
@JsonProperty("versionReplacementString") @Nullable String versionReplacementString,
@JsonProperty("version") String version
)
{
@ -48,6 +52,7 @@ public class StringEC2UserData implements EC2UserData<StringEC2UserData>
return data;
}
@Nullable
@JsonProperty
public String getVersionReplacementString()
{
@ -71,7 +76,7 @@ public class StringEC2UserData implements EC2UserData<StringEC2UserData>
{
final String finalData;
if (versionReplacementString != null && version != null) {
finalData = data.replace(versionReplacementString, version);
finalData = StringUtils.replace(data, versionReplacementString, version);
} else {
finalData = data;
}
@ -93,25 +98,16 @@ public class StringEC2UserData implements EC2UserData<StringEC2UserData>
if (data != null ? !data.equals(that.data) : that.data != null) {
return false;
}
if (version != null ? !version.equals(that.version) : that.version != null) {
if (!Objects.equals(version, that.version)) {
return false;
}
if (versionReplacementString != null
? !versionReplacementString.equals(that.versionReplacementString)
: that.versionReplacementString != null) {
return false;
}
return true;
return Objects.equals(versionReplacementString, that.versionReplacementString);
}
@Override
public int hashCode()
{
int result = data != null ? data.hashCode() : 0;
result = 31 * result + (versionReplacementString != null ? versionReplacementString.hashCode() : 0);
result = 31 * result + (version != null ? version.hashCode() : 0);
return result;
return Objects.hash(data, versionReplacementString, version);
}
@Override

View File

@ -694,7 +694,7 @@ public class OverlordResource
if (state == null || "complete".equals(StringUtils.toLowerCase(state))) {
Duration duration = null;
if (interval != null) {
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
duration = theInterval.toDuration();
}
final List<TaskInfo<Task, TaskStatus>> taskInfoList =

View File

@ -81,7 +81,7 @@ public class MergeTaskBaseTest
"_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" +
"_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0",
StandardCharsets.UTF_8
).toString() +
) +
"_";
Assert.assertEquals(
desiredPrefix,

View File

@ -47,10 +47,12 @@ import org.junit.runners.Parameterized;
import java.util.Collection;
import java.util.List;
import java.util.regex.Pattern;
@RunWith(Parameterized.class)
public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper
{
private static final Pattern WORD = Pattern.compile("\\w+");
@Parameterized.Parameters(name = "{index}: requestPath={0}, requestMethod={1}, resourceFilter={2}")
public static Collection<Object[]> data()
@ -190,7 +192,7 @@ public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper
@Test
public void testDatasourcesResourcesFilteringBadPath()
{
final String badRequestPath = requestPath.replaceAll("\\w+", "droid");
final String badRequestPath = WORD.matcher(requestPath).replaceAll("droid");
EasyMock.expect(request.getPath()).andReturn(badRequestPath).anyTimes();
EasyMock.replay(req, request, authorizerMapper);
Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath));

View File

@ -165,7 +165,8 @@ public class CoordinatorResourceTestClient
StringUtils.format(
"%sdatasources/%s/intervals/%s",
getCoordinatorURL(),
dataSource, interval.toString().replace("/", "_")
dataSource,
interval.toString().replace('/', '_')
)
);
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.tests.hadoop;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.testing.IntegrationTestingConfig;
import org.apache.druid.testing.guice.DruidTestModuleFactory;
@ -65,7 +66,7 @@ public class ITHadoopIndexTest extends AbstractIndexerTest
try {
LOG.info("indexerFile name: [%s]", BATCH_TASK);
indexerSpec = getTaskAsString(BATCH_TASK);
indexerSpec = indexerSpec.replaceAll("%%HADOOP_TEST_PATH%%", hadoopDir);
indexerSpec = StringUtils.replace(indexerSpec, "%%HADOOP_TEST_PATH%%", hadoopDir);
}
catch (Exception e) {
LOG.error("could not read and modify indexer file: %s", e.getMessage());

View File

@ -107,17 +107,17 @@ public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTes
}
query_response_template = IOUtils.toString(is, "UTF-8");
String queryStr = query_response_template
.replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast))
.replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst))
.replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)))
.replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%POST_AG_REQUEST_START%%", INTERVAL_FMT.print(dtFirst))
.replace("%%POST_AG_REQUEST_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)))
.replace("%%POST_AG_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtGroupBy.withSecondOfMinute(0))
);
String queryStr = query_response_template;
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_START%%", INTERVAL_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)));
String postAgResponseTimestamp = TIMESTAMP_FMT.print(dtGroupBy.withSecondOfMinute(0));
queryStr = StringUtils.replace(queryStr, "%%POST_AG_RESPONSE_TIMESTAMP%%", postAgResponseTimestamp);
// should hit the queries all on realtime task or some on realtime task
// and some on historical. Which it is depends on where in the minute we were
@ -158,7 +158,7 @@ public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTes
String setShutOffTime(String taskAsString, DateTime time)
{
return taskAsString.replace("#SHUTOFFTIME", time.toString());
return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString());
}
String getRouterURL()

View File

@ -22,6 +22,7 @@ import com.google.common.base.Throwables;
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient;
@ -99,7 +100,7 @@ public class ITAppenderatorDriverRealtimeIndexTaskTest extends AbstractITRealtim
} else if (i == 18) { // use a time 6 seconds ago so it will be out of order
dt = dt.minusSeconds(6);
}
String event = line.replace(TIME_PLACEHOLDER, EVENT_FMT.print(dt));
String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt));
LOG.info("sending event: [%s]\n", event);
Collection<Map<String, Object>> events = new ArrayList<Map<String, Object>>();
events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));

View File

@ -20,6 +20,7 @@
package org.apache.druid.tests.indexer;
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.druid.testing.guice.DruidTestModuleFactory;
import org.apache.druid.testing.utils.RetryUtil;
@ -95,7 +96,8 @@ public class ITCompactionTaskTest extends AbstractIndexerTest
private void compactData(boolean keepSegmentGranularity) throws Exception
{
final String template = getTaskAsString(COMPACTION_TASK);
final String taskSpec = template.replace("${KEEP_SEGMENT_GRANULARITY}", Boolean.toString(keepSegmentGranularity));
final String taskSpec =
StringUtils.replace(template, "${KEEP_SEGMENT_GRANULARITY}", Boolean.toString(keepSegmentGranularity));
final String taskID = indexer.submitTask(taskSpec);
LOG.info("TaskID for compaction task %s", taskID);
indexer.waitUntilTaskCompletes(taskID);

View File

@ -142,10 +142,10 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
consumerProperties.put("bootstrap.servers", config.getKafkaInternalHost());
addFilteredProperties(consumerProperties);
spec = getTaskAsString(INDEXER_FILE)
.replaceAll("%%DATASOURCE%%", DATASOURCE)
.replaceAll("%%TOPIC%%", TOPIC_NAME)
.replaceAll("%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties));
spec = getTaskAsString(INDEXER_FILE);
spec = StringUtils.replace(spec, "%%DATASOURCE%%", DATASOURCE);
spec = StringUtils.replace(spec, "%%TOPIC%%", TOPIC_NAME);
spec = StringUtils.replace(spec, "%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties));
LOG.info("supervisorSpec: [%s]\n", spec);
}
catch (Exception e) {
@ -227,16 +227,16 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
throw new ISE(e, "could not read query file: %s", QUERIES_FILE);
}
String queryStr = query_response_template
.replaceAll("%%DATASOURCE%%", DATASOURCE)
.replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast))
.replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst))
.replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)))
.replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMESERIES_ADDED%%", Integer.toString(added))
.replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events));
String queryStr = query_response_template;
queryStr = StringUtils.replace(queryStr, "%%DATASOURCE%%", DATASOURCE);
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_ADDED%%", Integer.toString(added));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events));
// this query will probably be answered from the indexing tasks but possibly from 2 historical segments / 2 indexing
try {

View File

@ -203,11 +203,12 @@ public class ITKafkaTest extends AbstractIndexerTest
addFilteredProperties(consumerProperties);
indexerSpec = getTaskAsString(INDEXER_FILE)
.replaceAll("%%DATASOURCE%%", DATASOURCE)
.replaceAll("%%TOPIC%%", TOPIC_NAME)
.replaceAll("%%COUNT%%", Integer.toString(num_events))
.replaceAll("%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties));
indexerSpec = getTaskAsString(INDEXER_FILE);
indexerSpec = StringUtils.replace(indexerSpec, "%%DATASOURCE%%", DATASOURCE);
indexerSpec = StringUtils.replace(indexerSpec, "%%TOPIC%%", TOPIC_NAME);
indexerSpec = StringUtils.replace(indexerSpec, "%%COUNT%%", Integer.toString(num_events));
String consumerPropertiesJson = jsonMapper.writeValueAsString(consumerProperties);
indexerSpec = StringUtils.replace(indexerSpec, "%%CONSUMER_PROPERTIES%%", consumerPropertiesJson);
LOG.info("indexerFile: [%s]\n", indexerSpec);
}
@ -261,18 +262,19 @@ public class ITKafkaTest extends AbstractIndexerTest
throw new ISE(e, "could not read query file: %s", QUERIES_FILE);
}
String queryStr = queryResponseTemplate
.replaceAll("%%DATASOURCE%%", DATASOURCE)
// time boundary
.replace("%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast))
.replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst))
// time series
.replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst))
.replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2)))
.replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst))
.replace("%%TIMESERIES_ADDED%%", Integer.toString(added))
.replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events));
String queryStr = queryResponseTemplate;
queryStr = StringUtils.replace(queryStr, "%%DATASOURCE%%", DATASOURCE);
// time boundary
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast));
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst));
// time series
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst));
String queryEnd = INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", queryEnd);
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_ADDED%%", Integer.toString(added));
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events));
// this query will probably be answered from the realtime task
try {

View File

@ -23,6 +23,7 @@ import com.google.common.base.Throwables;
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient;
@ -111,7 +112,7 @@ public class ITRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest
} else if (i == 18) { // use a time 6 seconds ago so it will be out of order
dt = dt.minusSeconds(6);
}
String event = line.replace(TIME_PLACEHOLDER, EVENT_FMT.print(dt));
String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt));
LOG.info("sending event: [%s]\n", event);
Collection<Map<String, Object>> events = new ArrayList<Map<String, Object>>();
events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));

View File

@ -157,17 +157,17 @@ public class ITUnionQueryTest extends AbstractIndexerTest
private String setShutOffTime(String taskAsString, DateTime time)
{
return taskAsString.replace("#SHUTOFFTIME", time.toString());
return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString());
}
private String withDataSource(String taskAsString, String dataSource)
{
return taskAsString.replace(UNION_DATASOURCE, dataSource);
return StringUtils.replace(taskAsString, UNION_DATASOURCE, dataSource);
}
private String withServiceName(String taskAsString, String serviceName)
{
return taskAsString.replace(EVENT_RECEIVER_SERVICE_PREFIX, serviceName);
return StringUtils.replace(taskAsString, EVENT_RECEIVER_SERVICE_PREFIX, serviceName);
}
public void postEvents(int id) throws Exception

View File

@ -933,7 +933,7 @@
<plugin>
<groupId>de.thetaphi</groupId>
<artifactId>forbiddenapis</artifactId>
<version>2.3</version>
<version>2.6</version>
<configuration>
<failOnUnresolvableSignatures>false</failOnUnresolvableSignatures>
<bundledSignatures>

View File

@ -94,7 +94,7 @@ public class WrappedConciseBitmap implements MutableBitmap
@Override
public String toString()
{
return getClass().getSimpleName() + bitmap.toString();
return getClass().getSimpleName() + bitmap;
}
@Override

View File

@ -67,7 +67,7 @@ public class WrappedImmutableConciseBitmap implements ImmutableBitmap
@Override
public String toString()
{
return getClass().getSimpleName() + bitmap.toString();
return getClass().getSimpleName() + bitmap;
}
@Override

View File

@ -70,7 +70,7 @@ public class WrappedImmutableRoaringBitmap implements ImmutableBitmap
@Override
public String toString()
{
return getClass().getSimpleName() + bitmap.toString();
return getClass().getSimpleName() + bitmap;
}
@Override

View File

@ -180,7 +180,7 @@ public class WrappedRoaringBitmap implements MutableBitmap
@Override
public String toString()
{
return getClass().getSimpleName() + bitmap.toString();
return getClass().getSimpleName() + bitmap;
}
@Override

View File

@ -178,8 +178,7 @@ public class CascadeExtractionFn implements ExtractionFn
@Override
public String toString()
{
return "CascadeExtractionFn{" +
"extractionFns=[" + chainedExtractionFn.toString() + "]}";
return "CascadeExtractionFn{extractionFns=[" + chainedExtractionFn + "]}";
}
private static class ChainedExtractionFn

View File

@ -119,7 +119,7 @@ public final class StringRuntimeShape
private void appendByteBufferShape(ByteBuffer byteBuffer)
{
sb.append(" {order: ");
sb.append(byteBuffer.order().toString());
sb.append(byteBuffer.order());
sb.append('}');
}

View File

@ -83,7 +83,7 @@ public class BySegmentSearchResultValue extends SearchResultValue
return "BySegmentSearchResultValue{" +
"results=" + results +
", segmentId='" + segmentId + '\'' +
", interval='" + interval.toString() + '\'' +
", interval='" + interval + '\'' +
'}';
}
}

View File

@ -85,7 +85,7 @@ public class BySegmentTopNResultValue extends TopNResultValue implements BySegme
return "BySegmentTopNResultValue{" +
"results=" + results +
", segmentId='" + segmentId + '\'' +
", interval='" + interval.toString() + '\'' +
", interval='" + interval + '\'' +
'}';
}
}

View File

@ -100,9 +100,9 @@ public class BitmapBenchmark
protected static void printSizeStats(double density, String name)
{
System.out.println("");
System.out.println();
System.out.println("## " + name);
System.out.println("");
System.out.println();
System.out.printf(Locale.ENGLISH, " d = %06.5f | Concise | Roaring%n", density);
System.out.println("-------------|---------|---------");
System.out.printf(Locale.ENGLISH, "Count | %5d | %5d %n", conciseCount, roaringCount);

View File

@ -414,22 +414,22 @@ public abstract class BaseFilterTest
)
{
Assert.assertEquals(
"Cursor: " + filter.toString(),
"Cursor: " + filter,
expectedRows,
selectColumnValuesMatchingFilter(filter, "dim0")
);
Assert.assertEquals(
"Cursor with postFiltering: " + filter.toString(),
"Cursor with postFiltering: " + filter,
expectedRows,
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
);
Assert.assertEquals(
"Filtered aggregator: " + filter.toString(),
"Filtered aggregator: " + filter,
expectedRows.size(),
selectCountUsingFilteredAggregator(filter)
);
Assert.assertEquals(
"RowBasedColumnSelectorFactory: " + filter.toString(),
"RowBasedColumnSelectorFactory: " + filter,
expectedRows,
selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0")
);

View File

@ -58,7 +58,7 @@ public class CoordinatorClient
StringUtils.format(
"/druid/coordinator/v1/datasources/%s/intervals/%s/serverview?partial=%s",
dataSource,
interval.toString().replace("/", "_"),
interval.toString().replace('/', '_'),
incompleteOk
))
);

View File

@ -42,6 +42,6 @@ public class CuratorServiceUtils
*/
protected static String makeCanonicalServiceName(String serviceName)
{
return serviceName.replaceAll("/", ":");
return serviceName.replace('/', ':');
}
}

View File

@ -135,7 +135,7 @@ public class LocalDataSegmentPusher implements DataSegmentPusher
private String makeIntermediateDir()
{
return "intermediate_pushes/" + UUID.randomUUID().toString();
return "intermediate_pushes/" + UUID.randomUUID();
}
private long compressSegment(File dataSegmentFile, File dest) throws IOException

View File

@ -138,8 +138,9 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory<InputRowPar
if (chatHandlerProvider.isPresent()) {
log.info("Found chathandler of class[%s]", chatHandlerProvider.get().getClass().getName());
chatHandlerProvider.get().register(serviceName, firehose);
if (serviceName.contains(":")) {
chatHandlerProvider.get().register(serviceName.replaceAll(".*:", ""), firehose); // rofl
int lastIndexOfColon = serviceName.lastIndexOf(':');
if (lastIndexOfColon > 0) {
chatHandlerProvider.get().register(serviceName.substring(lastIndexOfColon + 1), firehose);
}
} else {
log.warn("No chathandler detected");

View File

@ -625,7 +625,7 @@ public class RealtimePlumber implements Plumber
Object metadata = null;
long latestCommitTime = 0;
for (File sinkDir : files) {
final Interval sinkInterval = Intervals.of(sinkDir.getName().replace("_", "/"));
final Interval sinkInterval = Intervals.of(sinkDir.getName().replace('_', '/'));
//final File[] sinkFiles = sinkDir.listFiles();
// To avoid reading and listing of "merged" dir
@ -921,14 +921,17 @@ public class RealtimePlumber implements Plumber
protected File computeCorruptedFileDumpDir(File persistDir, DataSchema schema)
{
return new File(
persistDir.getAbsolutePath()
.replace(schema.getDataSource(), "corrupted" + File.pathSeparator + schema.getDataSource())
StringUtils.replace(
persistDir.getAbsolutePath(),
schema.getDataSource(),
"corrupted" + File.pathSeparator + schema.getDataSource()
)
);
}
protected File computePersistDir(DataSchema schema, Interval interval)
{
return new File(computeBaseDir(schema), interval.toString().replace("/", "_"));
return new File(computeBaseDir(schema), interval.toString().replace('/', '_'));
}
/**

View File

@ -86,13 +86,13 @@ public enum ServerType
@JsonCreator
public static ServerType fromString(String type)
{
return ServerType.valueOf(StringUtils.toUpperCase(type).replace("-", "_"));
return ServerType.valueOf(StringUtils.toUpperCase(type).replace('-', '_'));
}
@Override
@JsonValue
public String toString()
{
return StringUtils.toLowerCase(name()).replace("_", "-");
return StringUtils.toLowerCase(name()).replace('_', '-');
}
}

View File

@ -243,7 +243,7 @@ public class DatasourcesResource
if (indexingServiceClient == null) {
return Response.ok(ImmutableMap.of("error", "no indexing service found")).build();
}
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
try {
indexingServiceClient.killSegments(dataSourceName, theInterval);
}
@ -336,7 +336,7 @@ public class DatasourcesResource
)
{
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
if (dataSource == null) {
return Response.noContent().build();
@ -629,7 +629,7 @@ public class DatasourcesResource
TimelineLookup<String, SegmentLoadInfo> timeline = serverInventoryView.getTimeline(
new TableDataSource(dataSourceName)
);
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
if (timeline == null) {
log.debug("No timeline found for datasource[%s]", dataSourceName);
return Response.ok(new ArrayList<ImmutableSegmentLoadInfo>()).build();

View File

@ -98,7 +98,7 @@ public class IntervalsResource
@Context final HttpServletRequest req
)
{
final Interval theInterval = Intervals.of(interval.replace("_", "/"));
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
final Set<ImmutableDruidDataSource> datasources = InventoryViewUtils.getSecuredDataSources(
req,
serverInventoryView,

View File

@ -21,6 +21,7 @@ package org.apache.druid.server.metrics;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.metrics.Monitor;
import org.apache.druid.query.DruidMetrics;
@ -115,9 +116,11 @@ public class MonitorsConfig
}
try {
for (String monitorName : monitorNames) {
String effectiveMonitorName = monitorName
.replace(METAMX_PACKAGE, APACHE_DRUID_PACKAGE_NAME)
.replace(IO_DRUID_PACKAGE, APACHE_DRUID_PACKAGE_NAME);
final String effectiveMonitorName = StringUtils.replace(
StringUtils.replace(monitorName, METAMX_PACKAGE, APACHE_DRUID_PACKAGE_NAME),
IO_DRUID_PACKAGE,
APACHE_DRUID_PACKAGE_NAME
);
if (!effectiveMonitorName.equals(monitorName)) {
log.warn(
"Deprecated Monitor class name[%s] found, please use name[%s] instead.",

View File

@ -146,7 +146,7 @@ public class CuratorTestBase
druidServer.getType().toString(),
druidServer.getTier(),
DateTimes.nowUtc().toString()
) + String.valueOf(batchCtr++)
) + (batchCtr++)
);

View File

@ -62,7 +62,7 @@ public class ServiceAnnouncerTest extends CuratorTestBase
public boolean apply(String input)
{
try {
return serviceDiscovery.queryForInstances(input.replaceAll("/", ":")).size() == 1;
return serviceDiscovery.queryForInstances(input.replace('/', ':')).size() == 1;
}
catch (Exception e) {
throw new ISE(

View File

@ -67,7 +67,7 @@ public class TestDerbyConnector extends DerbyConnector
public static String dbSafeUUID()
{
return UUID.randomUUID().toString().replace("-", "");
return StringUtils.removeChar(UUID.randomUUID().toString(), '-');
}
public String getJdbcUri()

View File

@ -39,6 +39,7 @@ import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.regex.Pattern;
public class LocalDataSegmentPusherTest
{
@ -124,8 +125,10 @@ public class LocalDataSegmentPusherTest
DataSegment segment = localDataSegmentPusher.push(dataSegmentFiles, dataSegment, true);
String path = segment.getLoadSpec().get("path").toString();
String matcher = ".*/ds/1970-01-01T00:00:00\\.000Z_1970-01-01T00:00:00\\.001Z/v1/0/[A-Za-z0-9-]{36}/index\\.zip";
Assert.assertTrue(path, path.matches(matcher));
Pattern pattern = Pattern.compile(
".*/ds/1970-01-01T00:00:00\\.000Z_1970-01-01T00:00:00\\.001Z/v1/0/[A-Za-z0-9-]{36}/index\\.zip"
);
Assert.assertTrue(path, pattern.matcher(path).matches());
Assert.assertTrue(new File(path).exists());
}

View File

@ -39,6 +39,7 @@ import org.apache.druid.guice.annotations.Self;
import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.guice.http.DruidHttpClientConfig;
import org.apache.druid.initialization.Initialization;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
@ -350,11 +351,11 @@ public class AsyncQueryForwardingServletTest extends BaseJettyTest
@Override
protected String rewriteURI(HttpServletRequest request, String scheme, String host)
{
String uri = super.rewriteURI(request, scheme, host).toString();
String uri = super.rewriteURI(request, scheme, host);
if (uri.contains("/druid/v2")) {
return URI.create(uri.replace("/druid/v2", "/default")).toString();
return URI.create(StringUtils.replace(uri, "/druid/v2", "/default")).toString();
}
return URI.create(uri.replace("/proxy", "")).toString();
return URI.create(StringUtils.replace(uri, "/proxy", "")).toString();
}
});
//NOTE: explicit maxThreads to workaround https://tickets.puppetlabs.com/browse/TK-152

View File

@ -538,7 +538,7 @@ public class DatasourcesResourceTest
public void testDeleteDataSourceSpecificInterval()
{
String interval = "2010-01-01_P1D";
Interval theInterval = Intervals.of(interval.replace("_", "/"));
Interval theInterval = Intervals.of(interval.replace('_', '/'));
IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class);
indexingServiceClient.killSegments("datasource1", theInterval);

View File

@ -46,10 +46,13 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.util.Collection;
import java.util.regex.Pattern;
@RunWith(Parameterized.class)
public class SecurityResourceFilterTest extends ResourceFilterTestHelper
{
private static final Pattern WORD = Pattern.compile("\\w+");
@Parameterized.Parameters
public static Collection<Object[]> data()
{
@ -127,7 +130,7 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
public void testResourcesFilteringBadPath()
{
EasyMock.replay(req, request, authorizerMapper);
final String badRequestPath = requestPath.replaceAll("\\w+", "droid");
final String badRequestPath = WORD.matcher(requestPath).replaceAll("droid");
Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath));
EasyMock.verify(req, request, authorizerMapper);
}

View File

@ -49,7 +49,7 @@ public class FileRequestLoggerTest
ObjectMapper objectMapper = new ObjectMapper();
DateTime dateTime = DateTimes.nowUtc();
File logDir = temporaryFolder.newFolder();
String actualLogString = dateTime.toString() + "\t" + HOST;
String actualLogString = dateTime + "\t" + HOST;
FileRequestLogger fileRequestLogger = new FileRequestLogger(objectMapper, scheduler, logDir);
fileRequestLogger.start();

View File

@ -1,69 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.cli.convert;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicBoolean;
/**
*/
public class PrefixRename implements PropertyConverter
{
private final String prefix;
private final String outputPrefix;
private final AtomicBoolean ran = new AtomicBoolean(false);
public PrefixRename(
String prefix,
String outputPrefix
)
{
this.prefix = prefix;
this.outputPrefix = outputPrefix;
}
@Override
public boolean canHandle(String property)
{
return property.startsWith(prefix) && !ran.get();
}
@Override
public Map<String, String> convert(Properties properties)
{
if (!ran.getAndSet(true)) {
Map<String, String> retVal = Maps.newLinkedHashMap();
for (String property : properties.stringPropertyNames()) {
if (property.startsWith(prefix)) {
retVal.put(property.replace(prefix, outputPrefix), properties.getProperty(property));
}
}
return retVal;
}
return ImmutableMap.of();
}
}

View File

@ -25,6 +25,7 @@ import org.apache.calcite.avatica.metrics.Histogram;
import org.apache.calcite.avatica.metrics.Meter;
import org.apache.calcite.avatica.metrics.MetricsSystem;
import org.apache.calcite.avatica.metrics.Timer;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
@ -178,6 +179,6 @@ public class AvaticaMonitor extends AbstractMonitor implements MetricsSystem
private String fullMetricName(final String name)
{
return name.replace("org.apache.calcite.avatica", "avatica").replace(".", "/");
return StringUtils.replace(name, "org.apache.calcite.avatica", "avatica").replace('.', '/');
}
}

View File

@ -56,6 +56,7 @@ import org.joda.time.format.ISODateTimeFormat;
import java.nio.charset.Charset;
import java.util.NavigableSet;
import java.util.regex.Pattern;
/**
* Utility functions for Calcite.
@ -77,6 +78,8 @@ public class Calcites
private static final Charset DEFAULT_CHARSET = Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
private static final Pattern TRAILING_ZEROS = Pattern.compile("\\.?0+$");
private Calcites()
{
// No instantiation.
@ -129,7 +132,7 @@ public class Calcites
}
}
builder.append("'");
return isPlainAscii ? builder.toString() : "U&" + builder.toString();
return isPlainAscii ? builder.toString() : "U&" + builder;
}
@ -248,7 +251,10 @@ public class Calcites
public static TimestampString jodaToCalciteTimestampString(final DateTime dateTime, final DateTimeZone timeZone)
{
// The replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
return new TimestampString(CALCITE_TIMESTAMP_PRINTER.print(dateTime.withZone(timeZone)).replaceAll("\\.?0+$", ""));
String timestampString = TRAILING_ZEROS
.matcher(CALCITE_TIMESTAMP_PRINTER.print(dateTime.withZone(timeZone)))
.replaceAll("");
return new TimestampString(timestampString);
}
/**
@ -262,7 +268,10 @@ public class Calcites
public static TimeString jodaToCalciteTimeString(final DateTime dateTime, final DateTimeZone timeZone)
{
// The replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
return new TimeString(CALCITE_TIME_PRINTER.print(dateTime.withZone(timeZone)).replaceAll("\\.?0+$", ""));
String timeString = TRAILING_ZEROS
.matcher(CALCITE_TIME_PRINTER.print(dateTime.withZone(timeZone)))
.replaceAll("");
return new TimeString(timeString);
}
/**

View File

@ -465,7 +465,7 @@ public class SystemSchema extends AbstractSchema
authorizerMapper
);
if (!access.isAllowed()) {
throw new ForbiddenException("Insufficient permission to view servers :" + access.toString());
throw new ForbiddenException("Insufficient permission to view servers :" + access);
}
final FluentIterable<Object[]> results = FluentIterable
.from(druidServers)

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