Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion .idea/inspectionProfiles/Druid.xml

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public Object generateRowValue()
int rowSize = schema.getRowSize();

if (nullProbability != null) {
Double randDouble = simpleRng.nextDouble();
double randDouble = simpleRng.nextDouble();
if (randDouble <= nullProbability) {
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public void reseedRandomGenerator(long seed)
public int sample()
{
double randomVal = realDist.sample();
Long longVal = Math.round(randomVal);
return longVal.intValue();
long longVal = Math.round(randomVal);
return (int) longVal;
}
}
5 changes: 5 additions & 0 deletions codestyle/checkstyle.xml
Original file line number Diff line number Diff line change
Expand Up @@ -97,5 +97,10 @@
<property name="illegalPattern" value="true"/>
<property name="message" value="Use Double.NEGATIVE_INFINITY"/>
</module>
<module name="Regexp">
<property name="format" value="com\.metamx\.common\.logger"/>
<property name="illegalPattern" value="true"/>
<property name="message" value="Use io.druid.java.util.common.logger.Logger instead"/>
</module>
</module>
</module>
2 changes: 1 addition & 1 deletion common/src/main/java/io/druid/collections/StupidPool.java
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,7 @@ private void impossibleOffsetFailed(T object, ObjectId objectId, Cleaner cleaner
cleaner.clean();
log.error(
new ISE("Queue offer failed"),
"Could not offer object [%s] back into the queue in [%s], objectId [%s]",
"Could not offer object [%s] back into the queue, objectId [%s]",
object,
objectId
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.StringUtils;
import twitter4j.ConnectionLifeCycleListener;
import twitter4j.GeoLocation;
import twitter4j.HashtagEntity;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,12 @@

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.emitter.core.Emitter;
import com.metamx.emitter.core.Event;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@
import com.google.common.io.CharStreams;
import com.google.common.io.Files;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.java.util.common.logger.Logger;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;

import java.io.File;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ public Firehose connect(
pullMessageService.start();
}
catch (MQClientException e) {
LOGGER.error("Failed to start DefaultMQPullConsumer", e);
LOGGER.error(e, "Failed to start DefaultMQPullConsumer");
throw new IOException("Failed to start RocketMQ client", e);
}

Expand Down Expand Up @@ -228,7 +228,7 @@ public boolean hasMore()
}
}
catch (MQClientException e) {
LOGGER.error("Failed to fetch consume offset for queue: {}", entry.getKey());
LOGGER.error("Failed to fetch consume offset for queue: %s", entry.getKey());
}
}
}
Expand All @@ -241,7 +241,7 @@ public boolean hasMore()
hasMore = true;
}
catch (InterruptedException e) {
LOGGER.error("CountDownLatch await got interrupted", e);
LOGGER.error(e, "CountDownLatch await got interrupted");
}
}
return hasMore;
Expand Down Expand Up @@ -448,7 +448,7 @@ private void doPull()

case OFFSET_ILLEGAL:
LOGGER.error(
"Bad Pull Request: Offset is illegal. Offset used: {}",
"Bad Pull Request: Offset is illegal. Offset used: %d",
pullRequest.getNextBeginOffset()
);
break;
Expand All @@ -458,7 +458,7 @@ private void doPull()
}
}
catch (MQClientException | RemotingException | MQBrokerException | InterruptedException e) {
LOGGER.error("Failed to pull message from broker.", e);
LOGGER.error(e, "Failed to pull message from broker.");
}
finally {
pullRequest.getCountDownLatch().countDown();
Expand All @@ -485,7 +485,7 @@ public void run()
Thread.sleep(10);
}
catch (InterruptedException e) {
LOGGER.error("", e);
LOGGER.error(e, "");
}

synchronized (this) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public void getSegmentFiles(final DataSegment segment, final File outDir) throws
public FileUtils.FileCopyResult getSegmentFiles(final String bucket, final String path, File outDir)
throws SegmentLoadingException
{
LOG.info("Pulling index at path[%s] to outDir[%s]", bucket, path, outDir.getAbsolutePath());
LOG.info("Pulling index at bucket[%s] path[%s] to outDir[%s]", bucket, path, outDir.getAbsolutePath());

try {
prepareOutDir(outDir);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ public Iterable<BytesMessageWithOffset> fetch(long offset, int timeoutMs) throws
}
catch (Exception e) {
ensureNotInterrupted(e);
log.warn(e, "caught exception in fetch {} - {}", topic, partitionId);
log.warn(e, "caught exception in fetch %s - %d", topic, partitionId);
response = null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ public Sequence<ScanResultValue> process(
}
}
final boolean hasTimeout = QueryContexts.hasTimeout(query);
final Long timeoutAt = (long) responseContext.get(ScanQueryRunnerFactory.CTX_TIMEOUT_AT);
final long timeoutAt = (long) responseContext.get(ScanQueryRunnerFactory.CTX_TIMEOUT_AT);
final long start = System.currentTimeMillis();
final StorageAdapter adapter = segment.asStorageAdapter();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@

import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.metadata.SQLMetadataConnector;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public void aggregate(ByteBuffer buf, int position)
{
Long newTime = TimestampAggregatorFactory.convertLong(timestampSpec, selector.get());
if (newTime != null) {
Long prev = buf.getLong(position);
long prev = buf.getLong(position);
buf.putLong(position, comparator.compare(prev, newTime) > 0 ? prev: newTime);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,9 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
//CHECKSTYLE.OFF: Regexp
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why needs to use old logger?

import com.metamx.common.logger.Logger;
//CHECKSTYLE.ON: Regexp
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.LoggingEmitter;
import com.metamx.emitter.service.ServiceEmitter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,11 +167,11 @@ public void run()
// Won't hurt in remote mode, and is required for setting up locks in local mode:
try {
if (!task.isReady(taskActionClientFactory.create(task))) {
throw new ISE("Task is not ready to run yet!", task.getId());
throw new ISE("Task[%s] is not ready to run yet!", task.getId());
}
}
catch (Exception e) {
throw new ISE(e, "Failed to run isReady", task.getId());
throw new ISE(e, "Failed to run task[%s] isReady", task.getId());
}

statusFuture = Futures.transform(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,9 @@
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
//CHECKSTYLE.OFF: Regexp
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why needs to use old logger?

import com.metamx.common.logger.Logger;
//CHECKSTYLE.ON: Regexp
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.LoggingEmitter;
import com.metamx.emitter.service.ServiceEmitter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ public void afterClass()
unloadAndKillData(BATCH_DATASOURCE);
}
catch (Exception e) {
LOG.warn(e, "exception while removing segments: [%s]");
LOG.warn(e, "exception while removing segments");
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ public void testCharsetShowsUpAsDeprecated()
Assert.assertNotNull(StringUtils.UTF8_CHARSET);
}

@SuppressWarnings("MalformedFormatString")
@Test
public void testNonStrictFormat()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@

public class LoggerTest
{
@SuppressWarnings("MalformedFormatString")
@Test
public void testLogWithCrazyMessages()
{
Expand All @@ -31,6 +32,7 @@ public void testLogWithCrazyMessages()
log.warn(message);
}

@SuppressWarnings("MalformedFormatString")
@Test
public void testLegacyLogging()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@ public static <T> Sequence<Result<T>> makeCursorBasedQuery(
@Override
public Result<T> apply(Cursor input)
{
log.debug("Running over cursor[%s]", adapter.getInterval(), input.getTime());
return mapFn.apply(input);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ public void close()
aggregator.close();
}
catch (Exception e) {
log.warn(e, "Could not close aggregator, skipping.", aggregator);
log.warn(e, "Could not close aggregator [%s], skipping.", aggregator);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

package io.druid.query.groupby.resource;

import com.metamx.common.logger.Logger;
import io.druid.collections.ResourceHolder;
import io.druid.java.util.common.logger.Logger;

import java.io.Closeable;
import java.nio.ByteBuffer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ public Sequence<Result<TopNResultValue>> query(
@Override
public Result<TopNResultValue> apply(Cursor input)
{
log.debug("Running over cursor[%s]", adapter.getInterval(), input.getTime());
if (queryMetrics != null) {
queryMetrics.cursor(input);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,7 @@ protected Integer addToFacts(
throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount);
}

final Integer rowIndex = indexIncrement.getAndIncrement();
final int rowIndex = indexIncrement.getAndIncrement();

// note that indexAndOffsets must be updated before facts, because as soon as we update facts
// concurrent readers get hold of it and might ask for newly added row
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ protected Integer addToFacts(
factorizeAggs(metrics, aggs, rowContainer, row);
doAggregate(metrics, aggs, rowContainer, row, reportParseExceptions);

final Integer rowIndex = indexIncrement.getAndIncrement();
final int rowIndex = indexIncrement.getAndIncrement();
concurrentSet(rowIndex, aggs);

// Last ditch sanity checks
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ public byte[] put(ByteBuffer key, byte[] value)
if (logEvictions && evictionCount.get() % logEvictionCount == 0) {
log.info(
"Evicting %,dth element. Size[%,d], numBytes[%,d], averageSize[%,d]",
evictionCount,
evictionCount.get(),
size(),
numBytes.get(),
numBytes.get() / size()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ void checkForSegmentHandoffs()
catch (Throwable t) {
log.error(
t,
"Exception while checking handoff for dataSource[%s] Segment[%s], Will try again after [%d]secs",
"Exception while checking handoff for dataSource[%s], Will try again after [%d]secs",
dataSource,
pollDurationMillis
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ public DataSegment apply(DataSegment input)
public void announceSegment(DataSegment segment) throws IOException
{
if (segmentLookup.containsKey(segment)) {
log.info("Skipping announcement of segment [%s]. Announcement exists already.");
log.info("Skipping announcement of segment [%s]. Announcement exists already.", segment.getIdentifier());
return;
}

Expand Down Expand Up @@ -223,7 +223,7 @@ public void announceSegments(Iterable<DataSegment> segments) throws IOException
for (DataSegment ds : segments) {

if (segmentLookup.containsKey(ds)) {
log.info("Skipping announcement of segment [%s]. Announcement exists already.");
log.info("Skipping announcement of segment [%s]. Announcement exists already.", ds.getIdentifier());
return;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ public void unregister(String serviceName)
{
log.info("Unregistering EventReceiverFirehoseMetric for service [%s]", serviceName);
if (metrics.remove(serviceName) == null) {
log.warn("Unregistering a non-exist service. Service [%s] never exists.");
log.warn("Unregistering a non-exist service. Service [%s] never exists.", serviceName);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
//CHECKSTYLE.OFF: Regexp
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why needs to use old logger?

import com.metamx.common.logger.Logger;
//CHECKSTYLE.ON: Regexp
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.LoggingEmitter;
import com.metamx.emitter.service.ServiceEmitter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
//CHECKSTYLE.OFF: Regexp
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why needs to use old logger?

import com.metamx.common.logger.Logger;
//CHECKSTYLE.ON: Regexp
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.LoggingEmitter;
import com.metamx.emitter.service.ServiceEmitter;
Expand Down