Remove implied profanity from error messages. (#10270)

i.e. WTF, WTH.
This commit is contained in:
Gian Merlino 2020-08-28 11:38:50 -07:00 committed by GitHub
parent 5cd7610fb6
commit 8ab1979304
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
76 changed files with 149 additions and 140 deletions

View File

@ -29,6 +29,7 @@ import org.skife.config.ConfigurationObjectFactory;
import java.util.Map;
/**
*
*/
public class ConfigProvider<T> implements Provider<T>
{
@ -79,7 +80,7 @@ public class ConfigProvider<T> implements Provider<T>
{
try {
// ConfigMagic handles a null replacements
Preconditions.checkNotNull(factory, "WTF!? Code misconfigured, inject() didn't get called.");
Preconditions.checkNotNull(factory, "Code misconfigured, inject() didn't get called.");
return factory.buildWithReplacements(clazz, replacements);
}
catch (IllegalArgumentException e) {

View File

@ -205,7 +205,7 @@ public class FileSmoosher implements Closeable
bytesWritten += bytesWrittenInChunk;
if (bytesWritten != currOut.getCurrOffset() - startOffset) {
throw new ISE("WTF? Perhaps there is some concurrent modification going on?");
throw new ISE("Perhaps there is some concurrent modification going on?");
}
if (bytesWritten > size) {
throw new ISE("Wrote[%,d] bytes for something of size[%,d]. Liar!!!", bytesWritten, size);
@ -228,7 +228,7 @@ public class FileSmoosher implements Closeable
writerCurrentlyInUse = false;
if (bytesWritten != currOut.getCurrOffset() - startOffset) {
throw new ISE("WTF? Perhaps there is some concurrent modification going on?");
throw new ISE("Perhaps there is some concurrent modification going on?");
}
if (bytesWritten != size) {
throw new IOE("Expected [%,d] bytes, only saw [%,d], potential corruption?", size, bytesWritten);

View File

@ -162,16 +162,6 @@ public class Logger
log.error("ASSERTION_ERROR: " + message, formatArgs);
}
public void wtf(String message, Object... formatArgs)
{
error(message, formatArgs);
}
public void wtf(Throwable t, String message, Object... formatArgs)
{
error(t, message, formatArgs);
}
public void debugSegments(@Nullable final Collection<DataSegment> segments, @Nullable String preamble)
{
if (log.isDebugEnabled()) {

View File

@ -224,7 +224,7 @@ public class ResourcePool<K, V> implements Closeable
deficit--;
poolVal = null;
} else {
throw new IllegalStateException("WTF?! No objects left, and no object deficit. This is probably a bug.");
throw new IllegalStateException("Unexpected state: No objects left, and no object deficit");
}
}

View File

@ -151,7 +151,7 @@ public class SequenceInputStreamResponseHandler implements HttpResponseHandler<I
}
catch (IOException e) {
// This should never happen
log.wtf(e, "The empty stream threw an IOException");
log.error(e, "The empty stream threw an IOException");
throw new RuntimeException(e);
}
finally {

View File

@ -287,6 +287,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
* @param minorVersion the minor version to check overshadow relation. The found groups will have lower minor versions
* than this.
* @param fromState the state to search for overshadowed groups.
*
* @return a list of found atomicUpdateGroups. It could be empty if no groups are found.
*/
@VisibleForTesting
@ -333,6 +334,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
* @param minorVersion the minor version to check overshadow relation. The found groups will have higher minor
* versions than this.
* @param fromState the state to search for overshadowed groups.
*
* @return a list of found atomicUpdateGroups. It could be empty if no groups are found.
*/
@VisibleForTesting
@ -530,7 +532,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
findOvershadows(group, State.STANDBY)
);
if (overshadowingStandbys.isEmpty()) {
throw new ISE("WTH? atomicUpdateGroup[%s] is in overshadowed state, but no one overshadows it?", group);
throw new ISE("Unexpected state: atomicUpdateGroup[%s] is overshadowed, but nothing overshadows it", group);
}
groupsOvershadowingAug = overshadowingStandbys;
isOvershadowingGroupsFull = false;
@ -585,6 +587,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
* @param groups atomicUpdateGroups sorted by their rootPartitionRange
* @param startRootPartitionId the start partitionId of the root partition range to check the coverage
* @param endRootPartitionId the end partitionId of the root partition range to check the coverage
*
* @return true if the given groups fully cover the given partition range.
*/
private boolean doGroupsFullyCoverPartitionRange(
@ -675,7 +678,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
// If this chunk is already in the atomicUpdateGroup, it should be in knownPartitionChunks
// and this code must not be executed.
throw new ISE(
"WTH? chunk[%s] is in the atomicUpdateGroup[%s] but not in knownPartitionChunks[%s]?",
"Unexpected state: chunk[%s] is in the atomicUpdateGroup[%s] but not in knownPartitionChunks[%s]",
chunk,
atomicUpdateGroup,
knownPartitionChunks
@ -875,7 +878,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
if (!removed.equals(aug)) {
throw new ISE(
"WTH? actually removed atomicUpdateGroup[%s] is different from the one which is supposed to be[%s]",
"Unexpected state: Removed atomicUpdateGroup[%s] is different from expected atomicUpdateGroup[%s]",
removed,
aug
);
@ -896,7 +899,7 @@ class OvershadowableManager<T extends Overshadowable<T>>
if (!knownChunk.equals(partitionChunk)) {
throw new ISE(
"WTH? Same partitionId[%d], but known partition[%s] is different from the input partition[%s]",
"Unexpected state: Same partitionId[%d], but known partition[%s] is different from the input partition[%s]",
partitionChunk.getChunkNumber(),
knownChunk,
partitionChunk
@ -932,7 +935,8 @@ class OvershadowableManager<T extends Overshadowable<T>>
(SingleEntryShort2ObjectSortedMap<AtomicUpdateGroup<T>>) map;
//noinspection ConstantConditions
return singleMap.val.isFull();
});
}
);
}
@Nullable

View File

@ -101,7 +101,7 @@ public class IdUtilsTest
{
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("thingToValidate cannot contain whitespace character except space.");
IdUtils.validateId(THINGO, "wtf\u000Bis line tabulation");
IdUtils.validateId(THINGO, "what\u000Bis line tabulation");
}
@Test

View File

@ -107,7 +107,7 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements
public void emit(Event event)
{
if (!started.get()) {
throw new ISE("WTF emit was called while service is not started yet");
throw new ISE("Emit called unexpectedly before service start");
}
if (event instanceof ServiceMetricEvent) {
final TimelineMetric timelineEvent = timelineMetricConverter.druidEventToTimelineMetric((ServiceMetricEvent) event);

View File

@ -99,7 +99,7 @@ public class GraphiteEmitter implements Emitter
public void emit(Event event)
{
if (!started.get()) {
throw new ISE("WTF emit was called while service is not started yet");
throw new ISE("Emit called unexpectedly before service start");
}
if (event instanceof ServiceMetricEvent) {
final GraphiteEvent graphiteEvent = graphiteEventConverter.druidEventToGraphite((ServiceMetricEvent) event);

View File

@ -66,7 +66,7 @@ public class OpentsdbEmitter implements Emitter
public void emit(Event event)
{
if (!started.get()) {
throw new ISE("WTF emit was called while service is not started yet");
throw new ISE("Emit called unexpectedly before service start");
}
if (event instanceof ServiceMetricEvent) {
OpentsdbEvent opentsdbEvent = converter.convert((ServiceMetricEvent) event);

View File

@ -206,7 +206,7 @@ public class CoordinatorPollingBasicAuthorizerCacheManager implements BasicAutho
}
}
catch (Exception e) {
LOG.makeAlert(e, "WTF? Could not deserialize user/role map received from coordinator.").emit();
LOG.makeAlert(e, "Could not deserialize user/role map received from coordinator").emit();
}
}

View File

@ -39,7 +39,7 @@ public final class BloomFilterMergeAggregator
ByteBuffer other = selector.getObject();
if (other == null) {
// nulls should be empty bloom filters by this point, so encountering a nil column in merge agg is unexpected
throw new ISE("WTF?! Unexpected null value in BloomFilterMergeAggregator");
throw new ISE("Unexpected null value in BloomFilterMergeAggregator");
}
BloomKFilter.mergeBloomFilterByteBuffers(buf, buf.position(), other, other.position());
}

View File

@ -76,7 +76,7 @@ public class BloomFilterMergeAggregatorFactory extends BloomFilterAggregatorFact
final BaseNullableColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
// null columns should be empty bloom filters by this point, so encountering a nil column in merge agg is unexpected
if (selector instanceof NilColumnValueSelector) {
throw new ISE("WTF?! Unexpected NilColumnValueSelector");
throw new ISE("Unexpected NilColumnValueSelector");
}
return new BloomFilterMergeAggregator((ColumnValueSelector<ByteBuffer>) selector, getMaxNumEntries(), true);
}

View File

@ -2594,10 +2594,10 @@ public class KafkaSupervisorTest extends EasyMockSupport
}
Assert.assertTrue(
serviceEmitter.getStackTrace().startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")
serviceEmitter.getStackTrace().startsWith("org.apache.druid.java.util.common.ISE: Cannot find")
);
Assert.assertEquals(
"WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
"Cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
serviceEmitter.getExceptionMessage()
);
Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());

View File

@ -3194,9 +3194,9 @@ public class KinesisSupervisorTest extends EasyMockSupport
}
Assert.assertTrue(serviceEmitter.getStackTrace()
.startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find"));
.startsWith("org.apache.druid.java.util.common.ISE: Cannot find"));
Assert.assertEquals(
"WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
"Cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]",
serviceEmitter.getExceptionMessage()
);
Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());

View File

@ -119,7 +119,7 @@ public class PollingLookup extends LookupExtractor
}
final CacheRefKeeper cacheRefKeeper = refOfCacheKeeper.get();
if (cacheRefKeeper == null) {
throw new ISE("Cache reference is null WTF");
throw new ISE("Cache reference is null");
}
final PollingCache cache = cacheRefKeeper.getAndIncrementRef();
try {

View File

@ -239,7 +239,8 @@ public class DetermineHashedPartitionsJob implements Jobby
Map<String, Object> metrics = TaskMetricsUtils.makeIngestionRowMetrics(
jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_COUNTER).getValue(),
jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER).getValue(),
jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_PROCESSED_WITH_ERRORS_COUNTER)
.getValue(),
jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_UNPARSEABLE_COUNTER).getValue(),
jobCounters.findCounter(HadoopDruidIndexerConfig.IndexJobCounters.ROWS_THROWN_AWAY_COUNTER).getValue()
);
@ -318,7 +319,7 @@ public class DetermineHashedPartitionsJob implements Jobby
.bucketInterval(DateTimes.utc(inputRow.getTimestampFromEpoch()));
if (!maybeInterval.isPresent()) {
throw new ISE("WTF?! No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch());
throw new ISE("No bucket found for timestamp: %s", inputRow.getTimestampFromEpoch());
}
interval = maybeInterval.get();
}
@ -387,7 +388,7 @@ public class DetermineHashedPartitionsJob implements Jobby
Optional<Interval> intervalOptional = config.getGranularitySpec().bucketInterval(DateTimes.utc(key.get()));
if (!intervalOptional.isPresent()) {
throw new ISE("WTF?! No bucket found for timestamp: %s", key.get());
throw new ISE("No bucket found for timestamp: %s", key.get());
}
interval = intervalOptional.get();
}

View File

@ -464,7 +464,7 @@ public class DeterminePartitionsJob implements Jobby
final Optional<Interval> maybeInterval = config.getGranularitySpec().bucketInterval(timestamp);
if (!maybeInterval.isPresent()) {
throw new ISE("WTF?! No bucket found for timestamp: %s", timestamp);
throw new ISE("No bucket found for timestamp: %s", timestamp);
}
final Interval interval = maybeInterval.get();
@ -627,7 +627,7 @@ public class DeterminePartitionsJob implements Jobby
final long totalRows = firstDvc.numRows;
if (!"".equals(firstDvc.dim) || !"".equals(firstDvc.value)) {
throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!");
throw new IllegalStateException("Expected total row indicator on first k/v pair");
}
// "iterator" will now take us over many candidate dimensions

View File

@ -355,7 +355,7 @@ public class IndexGeneratorJob implements Jobby
final Optional<Bucket> bucket = getConfig().getBucket(inputRow);
if (!bucket.isPresent()) {
throw new ISE("WTF?! No bucket found for row: %s", inputRow);
throw new ISE("No bucket found for row: %s", inputRow);
}
final long truncatedTimestamp = granularitySpec.getQueryGranularity()

View File

@ -68,6 +68,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.NoopQueryRunner;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.realtime.FireDepartment;
@ -318,7 +319,10 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptySegments, segments, commitMetadata) -> {
if (mustBeNullOrEmptySegments != null && !mustBeNullOrEmptySegments.isEmpty()) {
throw new ISE("WTH? stream ingestion tasks are overwriting segments[%s]", mustBeNullOrEmptySegments);
throw new ISE(
"Stream ingestion task unexpectedly attempted to overwrite segments: %s",
SegmentUtils.commaSeparatedIdentifiers(mustBeNullOrEmptySegments)
);
}
final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.appendAction(
segments,

View File

@ -73,7 +73,7 @@ public class ArchiveTask extends AbstractFixedIntervalTask
for (final DataSegment unusedSegment : unusedSegments) {
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
"Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()

View File

@ -81,7 +81,7 @@ public class MoveTask extends AbstractFixedIntervalTask
for (final DataSegment unusedSegment : unusedSegments) {
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
"Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()

View File

@ -212,7 +212,7 @@ public class RealtimeIndexTask extends AbstractTask
runThread = Thread.currentThread();
if (this.plumber != null) {
throw new IllegalStateException("WTF?!? run with non-null plumber??!");
throw new IllegalStateException("Plumber must be null");
}
setupTimeoutAlert();

View File

@ -74,7 +74,7 @@ public class RestoreTask extends AbstractFixedIntervalTask
for (final DataSegment unusedSegment : unusedSegments) {
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
"Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getId(),
unusedSegment.getVersion(),
myLock.getVersion()

View File

@ -146,7 +146,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec, P extends PartitionL
final String mustBeNull = intervalToVersion.put(lock.getInterval(), lock.getVersion());
if (mustBeNull != null) {
throw new ISE(
"WTH? Two versions([%s], [%s]) for the same interval[%s]?",
"Unexpected state: Two versions([%s], [%s]) for the same interval[%s]",
lock.getVersion(),
mustBeNull,
lock.getInterval()

View File

@ -95,7 +95,7 @@ public abstract class BaseRestorableTaskRunner<WorkItemType extends TaskRunnerWo
final Task task = jsonMapper.readValue(taskFile, Task.class);
if (!task.getId().equals(taskId)) {
throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId());
throw new ISE("Task[%s] restore file had wrong id[%s]", taskId, task.getId());
}
if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) {

View File

@ -174,7 +174,7 @@ public class ForkingTaskRunner
final ForkingTaskRunnerWorkItem taskWorkItem = tasks.get(task.getId());
if (taskWorkItem == null) {
LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit();
LOGGER.makeAlert("TaskInfo disappeared!").addData("task", task.getId()).emit();
throw new ISE("TaskInfo disappeared for task[%s]!", task.getId());
}
@ -183,7 +183,7 @@ public class ForkingTaskRunner
}
if (taskWorkItem.processHolder != null) {
LOGGER.makeAlert("WTF?! TaskInfo already has a processHolder")
LOGGER.makeAlert("TaskInfo already has a processHolder")
.addData("task", task.getId())
.emit();
throw new ISE("TaskInfo already has processHolder for task[%s]!", task.getId());

View File

@ -770,7 +770,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
final RemoteTaskRunnerWorkItem removed = completeTasks.remove(taskId);
final Worker worker;
if (removed == null || (worker = removed.getWorker()) == null) {
log.makeAlert("WTF?! Asked to cleanup nonexistent task")
log.makeAlert("Asked to cleanup nonexistent task")
.addData("taskId", taskId)
.emit();
} else {
@ -901,7 +901,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
RemoteTaskRunnerWorkItem workItem = pendingTasks.remove(task.getId());
if (workItem == null) {
log.makeAlert("WTF?! Got a null work item from pending tasks?! How can this be?!")
log.makeAlert("Ignoring null work item from pending task queue")
.addData("taskId", task.getId())
.emit();
return false;
@ -1119,7 +1119,7 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
zkWorker.setWorker(worker);
} else {
log.warn(
"WTF, worker[%s] updated its announcement but we didn't have a ZkWorker for it. Ignoring.",
"Worker[%s] updated its announcement but we didn't have a ZkWorker for it. Ignoring.",
worker.getHost()
);
}

View File

@ -171,7 +171,7 @@ public class SingleTaskBackgroundRunner implements TaskRunner, QuerySegmentWalke
executorService.shutdown();
}
catch (SecurityException ex) {
log.wtf(ex, "I can't control my own threads!");
log.error(ex, "I can't control my own threads!");
}
}
@ -233,7 +233,7 @@ public class SingleTaskBackgroundRunner implements TaskRunner, QuerySegmentWalke
executorService.shutdownNow();
}
catch (SecurityException ex) {
log.wtf(ex, "I can't control my own threads!");
log.error(ex, "I can't control my own threads!");
}
}
}

View File

@ -146,7 +146,7 @@ public class TaskLockbox
final TaskLock savedTaskLock = Preconditions.checkNotNull(taskAndLock.rhs, "savedTaskLock");
if (savedTaskLock.getInterval().toDurationMillis() <= 0) {
// "Impossible", but you never know what crazy stuff can be restored from storage.
log.warn("WTF?! Got lock[%s] with empty interval for task: %s", savedTaskLock, task.getId());
log.warn("Ignoring lock[%s] with empty interval for task: %s", savedTaskLock, task.getId());
continue;
}

View File

@ -176,7 +176,7 @@ public class ThreadingTaskRunner
taskWorkItem = tasks.get(task.getId());
if (taskWorkItem == null) {
LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit();
LOGGER.makeAlert("TaskInfo disappeared").addData("task", task.getId()).emit();
throw new ISE("TaskInfo disappeared for task[%s]!", task.getId());
}

View File

@ -487,7 +487,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
long workerDiscoveryStartTime = System.currentTimeMillis();
while (!workerViewInitialized.await(30, TimeUnit.SECONDS)) {
if (System.currentTimeMillis() - workerDiscoveryStartTime > TimeUnit.MINUTES.toMillis(5)) {
throw new ISE("WTF! Couldn't discover workers.");
throw new ISE("Couldn't discover workers.");
} else {
log.info("Waiting for worker discovery...");
}
@ -1169,7 +1169,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
}
if (immutableWorker == null) {
throw new ISE("WTH! NULL immutableWorker");
throw new ISE("Unexpected state: null immutableWorker");
}
try {
@ -1405,7 +1405,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
break;
default:
log.makeAlert(
"WTF! Found unrecognized state[%s] of task[%s] in taskStorage. Notification[%s] from worker[%s] is ignored.",
"Found unrecognized state[%s] of task[%s] in taskStorage. Notification[%s] from worker[%s] is ignored.",
knownStatusInStorage.get().getStatusCode(),
taskId,
announcement,
@ -1468,7 +1468,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
break;
default:
log.makeAlert(
"WTF! Found unrecognized state[%s] of task[%s]. Notification[%s] from worker[%s] is ignored.",
"Found unrecognized state[%s] of task[%s]. Notification[%s] from worker[%s] is ignored.",
taskItem.getState(),
taskId,
announcement,
@ -1513,7 +1513,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
break;
default:
log.makeAlert(
"WTF! Found unrecognized state[%s] of task[%s]. Notification[%s] from worker[%s] is ignored.",
"Found unrecognized state[%s] of task[%s]. Notification[%s] from worker[%s] is ignored.",
taskItem.getState(),
taskId,
announcement,
@ -1523,7 +1523,7 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
break;
default:
log.makeAlert(
"WTF! Worker[%s] reported unrecognized state[%s] for task[%s].",
"Worker[%s] reported unrecognized state[%s] for task[%s].",
worker.getHost(),
announcement.getTaskStatus().getStatusCode(),
taskId

View File

@ -470,7 +470,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
// Sanity checks.
if (!restoredNextPartitions.getStream().equals(ioConfig.getStartSequenceNumbers().getStream())) {
throw new ISE(
"WTF?! Restored stream[%s] but expected stream[%s]",
"Restored stream[%s] but expected stream[%s]",
restoredNextPartitions.getStream(),
ioConfig.getStartSequenceNumbers().getStream()
);
@ -478,7 +478,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
if (!currOffsets.keySet().equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet())) {
throw new ISE(
"WTF?! Restored partitions[%s] but expected partitions[%s]",
"Restored partitions[%s] but expected partitions[%s]",
currOffsets.keySet(),
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().keySet()
);
@ -633,7 +633,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
if (sequenceToUse == null) {
throw new ISE(
"WTH?! cannot find any valid sequence for record with partition [%s] and sequenceNumber [%s]. Current sequences: %s",
"Cannot find any valid sequence for record with partition [%s] and sequenceNumber [%s]. Current sequences: %s",
record.getPartitionId(),
record.getSequenceNumber(),
sequences
@ -1616,7 +1616,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
pauseLock.lockInterruptibly();
// Perform all sequence related checks before checking for isPaused()
// and after acquiring pauseLock to correctly guard against duplicate requests
Preconditions.checkState(sequenceNumbers.size() > 0, "WTH?! No Sequences found to set end sequences");
Preconditions.checkState(sequenceNumbers.size() > 0, "No sequences found to set end sequences");
final SequenceMetadata<PartitionIdType, SequenceOffsetType> latestSequence = getLastSequenceMetadata();
final Set<PartitionIdType> exclusiveStartPartitions;
@ -1641,7 +1641,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
} else if (latestSequence.isCheckpointed()) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(StringUtils.format(
"WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]",
"Sequence [%s] has already endOffsets set, cannot set to [%s]",
latestSequence,
sequenceNumbers
)).build();

View File

@ -32,6 +32,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecor
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.timeline.DataSegment;
@ -341,7 +342,10 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
) throws IOException
{
if (mustBeNullOrEmptySegments != null && !mustBeNullOrEmptySegments.isEmpty()) {
throw new ISE("WTH? stream ingestion tasks are overwriting segments[%s]", mustBeNullOrEmptySegments);
throw new ISE(
"Stream ingestion task unexpectedly attempted to overwrite segments: %s",
SegmentUtils.commaSeparatedIdentifiers(mustBeNullOrEmptySegments)
);
}
final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata");
final SeekableStreamEndSequenceNumbers<PartitionIdType, SequenceOffsetType> finalPartitions =
@ -353,7 +357,7 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
// Sanity check, we should only be publishing things that match our desired end state.
if (!getEndOffsets().equals(finalPartitions.getPartitionSequenceNumberMap())) {
throw new ISE(
"WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].",
"Driver for sequence[%s] attempted to publish invalid metadata[%s].",
SequenceMetadata.this.toString(),
commitMetadata
);

View File

@ -423,7 +423,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId);
return false;
} else {
throw new ISE("WTH?! cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId,
throw new ISE("Cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId,
activelyReadingTaskGroups
);
}
@ -1494,7 +1494,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData());
if (prevTaskData != null) {
throw new ISE(
"WTH? a taskGroup[%s] already exists for new task[%s]",
"taskGroup[%s] already exists for new task[%s]",
prevTaskData,
taskId
);
@ -2518,7 +2518,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
// The below get should throw ExecutionException since result is null.
final Map<PartitionIdType, SequenceOffsetType> pauseResult = pauseFutures.get(i).get();
throw new ISE(
"WTH? The pause request for task [%s] is supposed to fail, but returned [%s]",
"Pause request for task [%s] should have failed, but returned [%s]",
taskId,
pauseResult
);
@ -2674,7 +2674,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
final String taskId = entry.getKey();
final TaskData taskData = entry.getValue();
Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId);
Preconditions.checkNotNull(taskData.status, "task[%s] has null status", taskId);
if (taskData.status.isFailure()) {
stateManager.recordCompletedTaskState(TaskState.FAILED);
@ -2774,7 +2774,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
continue;
}
Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId);
Preconditions.checkNotNull(taskData.status, "Task[%s] has null status", taskId);
// remove failed tasks
if (taskData.status.isFailure()) {

View File

@ -345,7 +345,7 @@ public abstract class WorkerTaskManager
if (taskId.equals(task.getId())) {
assignedTasks.put(taskId, task);
} else {
throw new ISE("WTF! Corrupted assigned task on disk[%s].", taskFile.getAbsoluteFile());
throw new ISE("Corrupted assigned task on disk[%s].", taskFile.getAbsoluteFile());
}
}
catch (IOException ex) {
@ -471,7 +471,7 @@ public abstract class WorkerTaskManager
if (taskId.equals(taskAnnouncement.getTaskId())) {
completedTasks.put(taskId, taskAnnouncement);
} else {
throw new ISE("WTF! Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile());
throw new ISE("Corrupted completed task on disk[%s].", taskFile.getAbsoluteFile());
}
}
catch (IOException ex) {
@ -699,7 +699,7 @@ public abstract class WorkerTaskManager
if (!status.isComplete()) {
log.warn(
"WTF?! Got status notice for task [%s] that isn't complete (status = [%s])...",
"Got status notice for task [%s] that isn't complete (status = [%s])...",
task.getId(),
status.getStatusCode()
);

View File

@ -331,7 +331,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
if (intervals.equals(ImmutableList.of(testCase.interval))) {
return ImmutableSet.copyOf(testCase.segments);
} else {
throw new IllegalArgumentException("WTF");
throw new IllegalArgumentException("BAD");
}
}

View File

@ -76,7 +76,7 @@ public class PropertiesModule implements Module
}
}
catch (FileNotFoundException e) {
log.wtf(e, "This can only happen if the .exists() call lied.");
log.error(e, "This can only happen if the .exists() call lied.");
}
finally {
CloseQuietly.close(stream);

View File

@ -124,7 +124,7 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
List<T> retVal = result.toList();
if (retVal == null) {
throw new ISE("Got a null list of results! WTF?!");
throw new ISE("Got a null list of results");
}
return retVal;

View File

@ -169,7 +169,7 @@ public class ByteBufferHashTable
}
if (newBuckets < maxBuckets) {
throw new ISE("WTF?! newBuckets[%,d] < maxBuckets[%,d]", newBuckets, maxBuckets);
throw new ISE("newBuckets[%,d] < maxBuckets[%,d]", newBuckets, maxBuckets);
}
ByteBuffer newTableBuffer = buffer.duplicate();
@ -206,7 +206,7 @@ public class ByteBufferHashTable
final int newBucket = findBucket(true, newBuckets, newTableBuffer, keyBuffer, keyHash);
if (newBucket < 0) {
throw new ISE("WTF?! Couldn't find a bucket while resizing?!");
throw new ISE("Couldn't find a bucket while resizing");
}
final int newBucketOffset = newBucket * bucketSizeWithHash;
@ -230,7 +230,7 @@ public class ByteBufferHashTable
growthCount++;
if (size != newSize) {
throw new ISE("WTF?! size[%,d] != newSize[%,d] after resizing?!", size, newSize);
throw new ISE("size[%,d] != newSize[%,d] after resizing", size, newSize);
}
}

View File

@ -98,7 +98,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
// We check this already in SpillingGrouper to ensure that LimitedBufferHashGrouper is only used when there is
// sufficient buffer capacity. If this error occurs, something went very wrong.
if (!validateBufferCapacity(totalBuffer.capacity())) {
throw new IAE("WTF? Using LimitedBufferHashGrouper with insufficient buffer capacity.");
throw new IAE("LimitedBufferHashGrouper initialized with insufficient buffer capacity");
}
//only store offsets up to `limit` + 1 instead of up to # of buckets, we only keep the top results
@ -485,7 +485,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
final int newBucket = findBucket(true, maxBuckets, newTableBuffer, keyBuffer, keyHash);
if (newBucket < 0) {
throw new ISE("WTF?! Couldn't find a bucket while resizing?!");
throw new ISE("Couldn't find a bucket while resizing");
}
final int newBucketOffset = newBucket * bucketSizeWithHash;

View File

@ -65,7 +65,7 @@ public class ScanQueryEngine
)
{
// "legacy" should be non-null due to toolChest.mergeResults
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "WTF?! Expected non-null legacy");
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
final Object numScannedRows = responseContext.get(ResponseContext.Key.NUM_SCANNED_ROWS);
if (numScannedRows != null) {

View File

@ -253,7 +253,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory<ScanResultValu
}
}
if (finalInterval == null) {
throw new ISE("Row unexpectedly came from an unscanned interval");
throw new ISE("Row came from an unscanned interval");
}
}
}

View File

@ -80,7 +80,7 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
AggregatorUtil.condensedAggregators(query.getAggregatorSpecs(), query.getPostAggregatorSpecs(), metric);
if (condensedAggPostAggPair.lhs.isEmpty() && condensedAggPostAggPair.rhs.isEmpty()) {
throw new ISE("WTF! Can't find the metric to do topN over?");
throw new ISE("Can't find the topN metric");
}
// Run topN for only a single metric
TopNQuery singleMetricQuery = new TopNQueryBuilder(query)

View File

@ -217,7 +217,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex<BufferAggregator>
if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) {
getNumEntries().incrementAndGet();
} else {
throw new ISE("WTF! we are in sychronized block.");
throw new ISE("Unexpected state: Concurrent fact addition.");
}
}
}

View File

@ -60,7 +60,7 @@ public class SingleLongInputCachingExpressionColumnValueSelector implements Colu
{
// Verify expression has just one binding.
if (expression.analyzeInputs().getRequiredBindings().size() != 1) {
throw new ISE("WTF?! Expected expression with just one binding");
throw new ISE("Expected expression with just one binding");
}
this.selector = Preconditions.checkNotNull(selector, "selector");

View File

@ -56,7 +56,7 @@ public class SingleStringInputCachingExpressionColumnValueSelector implements Co
{
// Verify expression has just one binding.
if (expression.analyzeInputs().getRequiredBindings().size() != 1) {
throw new ISE("WTF?! Expected expression with just one binding");
throw new ISE("Expected expression with just one binding");
}
this.selector = Preconditions.checkNotNull(selector, "selector");

View File

@ -210,7 +210,7 @@ public class SchemaEvolutionTest
if (index4.getAvailableDimensions().size() != 0) {
// Just double-checking that the exclusions are working properly
throw new ISE("WTF?! Expected no dimensions in index4");
throw new ISE("Expected no dimensions in index4");
}
}

View File

@ -132,7 +132,7 @@ public class LimitedBufferHashGrouperTest
public void testBufferTooSmall()
{
expectedException.expect(IAE.class);
expectedException.expectMessage("WTF? Using LimitedBufferHashGrouper with insufficient buffer capacity.");
expectedException.expectMessage("LimitedBufferHashGrouper initialized with insufficient buffer capacity");
final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory();
makeGrouper(columnSelectorFactory, 10, 2, 100);
}

View File

@ -60,7 +60,7 @@ public class ServerDiscoverySelector implements DiscoverySelector<Server>
{
Preconditions.checkState(
instance.getPort() >= 0 || (instance.getSslPort() != null && instance.getSslPort() >= 0),
"WTH?! Both port and sslPort not set"
"Both port and sslPort not set"
);
final int port;
final String scheme;

View File

@ -322,7 +322,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
// Metadata transaction cannot fail because we are not trying to do one.
if (!result.isSuccess()) {
throw new ISE("WTF?! announceHistoricalSegments failed with null metadata, should not happen.");
throw new ISE("announceHistoricalSegments failed with null metadata, should not happen.");
}
return result.getSegments();

View File

@ -334,9 +334,9 @@ public class SegmentLoaderLocalCacheManager implements SegmentLoader
dataSegment,
(segment, existingLock) -> {
if (existingLock == null) {
throw new ISE("WTH? the given lock has already been removed");
throw new ISE("Lock has already been removed");
} else if (existingLock != lock) {
throw new ISE("WTH? Different lock instance");
throw new ISE("Different lock instance");
} else {
if (existingLock.numReferences == 1) {
return null;

View File

@ -96,7 +96,7 @@ public class FireHydrant
!newSegment.getId().equals(currentSegment.getId())) {
// Sanity check: identifier should not change
throw new ISE(
"WTF?! Cannot swap identifier[%s] -> [%s]!",
"Cannot swap identifier[%s] -> [%s]",
currentSegment.getId(),
newSegment.getId()
);

View File

@ -706,12 +706,12 @@ public class AppenderatorImpl implements Appenderator
// Sanity checks
for (FireHydrant hydrant : sink) {
if (sink.isWritable()) {
throw new ISE("WTF?! Expected sink to be no longer writable before mergeAndPush. Segment[%s].", identifier);
throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier);
}
synchronized (hydrant) {
if (!hydrant.hasSwapped()) {
throw new ISE("WTF?! Expected sink to be fully persisted before mergeAndPush. Segment[%s].", identifier);
throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier);
}
}
}

View File

@ -131,7 +131,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
// There should be only one appending segment at any time
Preconditions.checkState(
this.appendingSegment == null,
"WTF?! Current appendingSegment[%s] is not null. "
"Current appendingSegment[%s] is not null. "
+ "Its state must be changed before setting a new appendingSegment[%s]",
this.appendingSegment,
appendingSegment
@ -345,7 +345,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
for (SegmentIdWithShardSpec identifier : appenderator.getSegments()) {
if (identifier.equals(newSegment)) {
throw new ISE(
"WTF?! Allocated segment[%s] which conflicts with existing segment[%s].",
"Allocated segment[%s] which conflicts with existing segment[%s].",
newSegment,
identifier
);
@ -418,7 +418,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
);
}
catch (SegmentNotWritableException e) {
throw new ISE(e, "WTF?! Segment[%s] not writable when it should have been.", identifier);
throw new ISE(e, "Segment[%s] not writable when it should have been.", identifier);
}
} else {
return AppenderatorDriverAddResult.fail();

View File

@ -197,7 +197,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
synchronized (segments) {
final SegmentsForSequence activeSegmentsForSequence = segments.get(sequenceName);
if (activeSegmentsForSequence == null) {
throw new ISE("WTF?! Asked to remove segments for sequenceName[%s] which doesn't exist...", sequenceName);
throw new ISE("Asked to remove segments for sequenceName[%s], which doesn't exist", sequenceName);
}
for (final SegmentIdWithShardSpec identifier : identifiers) {
@ -207,7 +207,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
if (segmentsOfInterval == null ||
segmentsOfInterval.getAppendingSegment() == null ||
!segmentsOfInterval.getAppendingSegment().getSegmentIdentifier().equals(identifier)) {
throw new ISE("WTF?! Asked to remove segment[%s] that didn't exist...", identifier);
throw new ISE("Asked to remove segment[%s], which doesn't exist", identifier);
}
segmentsOfInterval.finishAppendingToCurrentActiveSegment(SegmentWithState::finishAppending);
}
@ -424,7 +424,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
if (segmentWithState.getState() == SegmentState.APPENDING) {
if (pair != null && pair.lhs != null) {
throw new ISE(
"WTF?! there was already an appendingSegment[%s] before adding an appendingSegment[%s]",
"appendingSegment[%s] existed before adding an appendingSegment[%s]",
pair.lhs,
segmentWithState
);

View File

@ -76,7 +76,7 @@ public class Plumbers
catch (IndexSizeExceededException e) {
// Shouldn't happen if this is only being called by a single thread.
// plumber.add should be swapping out indexes before they fill up.
throw new ISE(e, "WTF?! Index size exceeded, this shouldn't happen. Bad Plumber!");
throw new ISE(e, "Index size exceeded");
}
if (addResult.getRowCount() == -1) {

View File

@ -95,6 +95,7 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
/**
*
*/
public class RealtimePlumber implements Plumber
{
@ -213,7 +214,8 @@ public class RealtimePlumber implements Plumber
}
@Override
public IncrementalIndexAddResult add(InputRow row, Supplier<Committer> committerSupplier) throws IndexSizeExceededException
public IncrementalIndexAddResult add(InputRow row, Supplier<Committer> committerSupplier)
throws IndexSizeExceededException
{
long messageTimestamp = row.getTimestampFromEpoch();
final Sink sink = getSink(messageTimestamp);
@ -394,7 +396,7 @@ public class RealtimePlumber implements Plumber
if (!isPushedMarker.exists()) {
removeSegment(sink, mergedTarget);
if (mergedTarget.exists()) {
log.wtf("Merged target[%s] exists?!", mergedTarget);
log.warn("Merged target[%s] still exists after attempt to delete it; skipping push.", mergedTarget);
return;
}
} else {

View File

@ -391,7 +391,7 @@ public class ChangeRequestHttpSyncer<T>
} else {
log.makeAlert(
th,
"WTF! Couldn't schedule next sync. [%s] is not being synced any more, restarting Druid process on that "
"Couldn't schedule next sync. [%s] is not being synced any more, restarting Druid process on that "
+ "server might fix the issue.",
logIdentity
).emit();

View File

@ -227,7 +227,7 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
break;
default:
scheduleNextRunImmediately = false;
log.error("WTF! Server[%s] returned unknown state in status[%s].", serverId, e.getStatus());
log.error("Server[%s] returned unknown state in status[%s].", serverId, e.getStatus());
}
}
}

View File

@ -98,7 +98,7 @@ public class CompactSegments implements CoordinatorDuty
for (TaskStatusPlus status : compactionTasks) {
final TaskPayloadResponse response = indexingServiceClient.getTaskPayload(status.getId());
if (response == null) {
throw new ISE("WTH? got a null paylord from overlord for task[%s]", status.getId());
throw new ISE("Got a null paylord from overlord for task[%s]", status.getId());
}
if (COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) {
final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload();
@ -107,7 +107,7 @@ public class CompactSegments implements CoordinatorDuty
final int numSubTasks = findNumMaxConcurrentSubTasks(compactionTaskQuery.getTuningConfig());
numEstimatedNonCompleteCompactionTasks += numSubTasks + 1; // count the compaction task itself
} else {
throw new ISE("WTH? task[%s] is not a compactionTask?", status.getId());
throw new ISE("task[%s] is not a compactionTask", status.getId());
}
}

View File

@ -395,7 +395,7 @@ public class LookupCoordinatorManager
lookupCoordinatorManagerConfig.getHostTimeout().getMillis() * 10,
TimeUnit.MILLISECONDS
)) {
throw new ISE("WTF! LookupCoordinatorManager executor from last start() hasn't finished. Failed to Start.");
throw new ISE("LookupCoordinatorManager executor from last start() hasn't finished. Failed to Start.");
}
executorService = MoreExecutors.listeningDecorator(

View File

@ -844,7 +844,7 @@ public class AppenderatorTest extends InitializedNullHandlingTest
} else if (a instanceof DataSegment && b instanceof DataSegment) {
return ((DataSegment) a).getId().compareTo(((DataSegment) b).getId());
} else {
throw new IllegalStateException("WTF??");
throw new IllegalStateException("BAD");
}
}
);

View File

@ -189,7 +189,7 @@ public class DumpSegment extends GuiceRunnable
runBitmaps(injector, index);
break;
default:
throw new ISE("WTF?! dumpType[%s] has no handler?", dumpType);
throw new ISE("dumpType[%s] has no handler", dumpType);
}
}
catch (Exception e) {

View File

@ -60,7 +60,7 @@ public class BinaryOperatorConversion implements SqlOperatorConversion
rexNode,
operands -> {
if (operands.size() < 2) {
throw new ISE("WTF?! Got binary operator[%s] with %s args?", operator.getName(), operands.size());
throw new ISE("Got binary operator[%s] with %s args", operator.getName(), operands.size());
}
return DruidExpression.fromExpression(
@ -92,7 +92,7 @@ public class BinaryOperatorConversion implements SqlOperatorConversion
rexNode,
operands -> {
if (operands.size() < 2) {
throw new ISE("WTF?! Got binary operator[%s] with %s args?", operator.getName(), operands.size());
throw new ISE("Got binary operator[%s] with %s args", operator.getName(), operands.size());
}
return DruidExpression.fromExpression(

View File

@ -221,7 +221,7 @@ public class Expressions
final RexInputRef ref = (RexInputRef) rexNode;
final String columnName = rowSignature.getColumnName(ref.getIndex());
if (columnName == null) {
throw new ISE("WTF?! Expression referred to nonexistent index[%d]", ref.getIndex());
throw new ISE("Expression referred to nonexistent index[%d]", ref.getIndex());
}
return DruidExpression.fromColumn(columnName);
@ -490,7 +490,7 @@ public class Expressions
|| kind == SqlKind.LESS_THAN
|| kind == SqlKind.LESS_THAN_OR_EQUAL) {
final List<RexNode> operands = ((RexCall) rexNode).getOperands();
Preconditions.checkState(operands.size() == 2, "WTF?! Expected 2 operands, got[%,d]", operands.size());
Preconditions.checkState(operands.size() == 2, "Expected 2 operands, got[%,d]", operands.size());
boolean flip = false;
RexNode lhs = operands.get(0);
RexNode rhs = operands.get(1);
@ -525,7 +525,7 @@ public class Expressions
flippedKind = SqlKind.GREATER_THAN_OR_EQUAL;
break;
default:
throw new ISE("WTF?! Kind[%s] not expected here", kind);
throw new ISE("Kind[%s] not expected here", kind);
}
} else {
flippedKind = kind;
@ -632,7 +632,7 @@ public class Expressions
filter = Bounds.lessThanOrEqualTo(boundRefKey, val);
break;
default:
throw new IllegalStateException("WTF?! Shouldn't have got here...");
throw new IllegalStateException("Shouldn't have got here");
}
return filter;
@ -770,7 +770,7 @@ public class Expressions
case LESS_THAN_OR_EQUAL:
return Bounds.lessThan(boundRefKey, String.valueOf(interval.getEndMillis()));
default:
throw new IllegalStateException("WTF?! Shouldn't have got here...");
throw new IllegalStateException("Shouldn't have got here");
}
}
}

View File

@ -195,7 +195,7 @@ public class OperatorConversions
final RexInputRef ref = (RexInputRef) rexNode;
final String columnName = rowSignature.getColumnName(ref.getIndex());
if (columnName == null) {
throw new ISE("WTF?! PostAgg referred to nonexistent index[%d]", ref.getIndex());
throw new ISE("PostAggregator referred to nonexistent index[%d]", ref.getIndex());
}
return new FieldAccessPostAggregator(

View File

@ -23,6 +23,7 @@ import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
@ -59,8 +60,7 @@ public class CeilOperatorConversion implements SqlOperatorConversion
TimeFloorOperatorConversion.toTimestampFloorOrCeilArgs(plannerContext, rowSignature, call.getOperands())
);
} else {
// WTF? CEIL with the wrong number of arguments?
return null;
throw new ISE("Unexpected number of arguments");
}
}
}

View File

@ -23,6 +23,7 @@ import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
@ -59,8 +60,7 @@ public class FloorOperatorConversion implements SqlOperatorConversion
TimeFloorOperatorConversion.toTimestampFloorOrCeilArgs(plannerContext, rowSignature, call.getOperands())
);
} else {
// WTF? FLOOR with the wrong number of arguments?
return null;
throw new ISE("Unexpected number of arguments");
}
}
}

View File

@ -76,7 +76,7 @@ public class BoundValue implements Comparable<BoundValue>
public int compareTo(BoundValue o)
{
if (!comparator.equals(o.comparator)) {
throw new ISE("WTF?! Comparator mismatch?!");
throw new ISE("Comparator mismatch");
}
return comparator.compare(value, o.value);
}

View File

@ -153,7 +153,8 @@ public class CombineAndSimplifyBounds extends BottomUpTransform
// We found a simplification. Remove the old filters and add new ones.
for (final BoundDimFilter bound : filterList) {
if (!newChildren.remove(bound)) {
throw new ISE("WTF?! Tried to remove bound but couldn't?");
// Don't expect this to happen, but include it as a sanity check.
throw new ISE("Tried to remove bound, but couldn't");
}
}

View File

@ -85,7 +85,8 @@ public class ConvertSelectorsToIns extends BottomUpTransform
for (final SelectorDimFilter selector : filterList) {
values.add(selector.getValue());
if (!children.remove(selector)) {
throw new ISE("WTF?! Tried to remove selector but couldn't?");
// Don't expect this to happen, but include it as a sanity check.
throw new ISE("Tried to remove selector but couldn't");
}
}

View File

@ -141,7 +141,8 @@ public class Filtration
);
if (!transformed.getIntervals().equals(ImmutableList.of(eternity()))) {
throw new ISE("WTF?! optimizeFilterOnly was about to return filtration with intervals?!");
// Should not happen, but include as a sanity check to be sure.
throw new ISE("optimizeFilterOnly was about to return filtration with intervals");
}
return transformed;

View File

@ -149,7 +149,7 @@ public class DruidConvertletTable implements SqlRexConvertletTable
)
);
} else {
throw new ISE("WTF?! Should not have got here, operator was: %s", operator);
throw new ISE("Should not have got here, operator was: %s", operator);
}
}
}

View File

@ -393,7 +393,7 @@ public class PartialDruidQuery
case SCAN:
return scan;
default:
throw new ISE("WTF?! Unknown stage: %s", currentStage);
throw new ISE("Unknown stage: %s", currentStage);
}
}

View File

@ -61,7 +61,7 @@ public class ProjectAggregatePruneUnusedCallRule extends RelOptRule
final int fieldCount = aggregate.getGroupCount() + aggregate.getAggCallList().size();
if (fieldCount != aggregate.getRowType().getFieldCount()) {
throw new ISE(
"WTF, expected[%s] to have[%s] fields but it had[%s]",
"Expected[%s] to have[%s] fields but it had[%s]",
aggregate,
fieldCount,
aggregate.getRowType().getFieldCount()