From e267f3901b47d93048f9c822f8f23530497b237f Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 21 Sep 2017 15:06:48 -0500 Subject: [PATCH] Enforce Indentation with Checkstyle (#4799) --- .../java/io/druid/data/input/FirehoseV2.java | 77 +- .../jackson/CommaListJoinDeserializer.java | 2 +- .../SingleElementPartitionChunk.java | 5 +- .../druid/guice/ConditionalMultibindTest.java | 57 +- .../indexing/IndexPersistBenchmark.java | 2 +- codestyle/checkstyle-suppressions.xml | 3 + codestyle/checkstyle.xml | 5 + .../druid/collections/CombiningIterable.java | 12 +- .../timeline/partition/PartitionHolder.java | 24 +- .../ambari/metrics/AmbariMetricsEmitter.java | 52 +- .../druid/storage/google/GoogleStorage.java | 2 +- .../google/GoogleStorageDruidModule.java | 2 +- .../input/orc/OrcIndexGeneratorJobTest.java | 14 +- .../io/druid/client/cache/RedisCache.java | 272 ++-- .../druid/client/cache/RedisCacheConfig.java | 94 +- .../client/cache/RedisCacheProvider.java | 10 +- .../druid/client/cache/RedisDruidModule.java | 23 +- .../io/druid/client/cache/RedisCacheTest.java | 275 ++-- .../druid/emitter/statsd/StatsDEmitter.java | 47 +- .../emitter/statsd/StatsDEmitterModule.java | 2 +- .../security/kerberos/KerberosHttpClient.java | 30 +- .../kerberos/ResponseCookieHandler.java | 4 +- .../RetryIfUnauthorizedResponseHandler.java | 9 +- .../kerberos/SpnegoFilterConfigTest.java | 30 +- .../hdfs/HdfsDataSegmentPusherTest.java | 34 +- .../histogram/ApproximateHistogram.java | 6 +- .../ApproximateHistogramTopNQueryTest.java | 10 +- .../lookup/KafkaLookupExtractorFactory.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 47 +- .../NamespaceLookupExtractorFactoryTest.java | 4 +- .../cache/JdbcExtractionNamespaceTest.java | 17 +- .../server/lookup/LookupExtractionModule.java | 2 +- .../cache/loading/OffHeapLoadingCache.java | 3 +- .../server/lookup/jdbc/JdbcDataFetcher.java | 22 +- .../cache/loading/LoadingCacheTest.java | 10 +- .../lookup/jdbc/JdbcDataFetcherTest.java | 3 +- .../postgresql/PostgreSQLConnector.java | 6 +- .../storage/s3/S3DataSegmentFinderTest.java | 3 +- .../indexer/DetermineHashedPartitionsJob.java | 18 +- .../java/io/druid/indexer/JobHelperTest.java | 3 +- .../MetadataStorageUpdaterJobSpecTest.java | 23 +- .../overlord/MetadataTaskStorage.java | 2 +- .../druid/indexing/overlord/TaskLockbox.java | 11 +- .../druid/indexing/overlord/TaskMaster.java | 2 +- .../executor/ExecutorLifecycleConfig.java | 10 +- .../indexing/common/TestRealtimeTask.java | 23 +- .../indexing/common/task/IndexTaskTest.java | 144 ++- .../common/task/RealtimeIndexTaskTest.java | 47 +- .../task/SameIntervalMergeTaskTest.java | 196 +-- .../indexing/common/task/TaskSerdeTest.java | 18 +- .../IngestSegmentFirehoseFactoryTest.java | 20 +- ...estSegmentFirehoseFactoryTimelineTest.java | 20 +- .../indexing/overlord/RealtimeishTask.java | 44 +- .../indexing/overlord/TaskLifecycleTest.java | 23 +- ...dingTaskBasedProvisioningStrategyTest.java | 29 +- .../SimpleProvisioningStrategyTest.java | 26 +- ...nWithAffinityWorkerSelectStrategyTest.java | 17 +- .../indexing/worker/TaskAnnouncementTest.java | 18 +- .../IntegrationTestingCuratorConfig.java | 2 +- .../CoordinatorResourceTestClient.java | 5 +- .../clients/OverlordResourceTestClient.java | 6 +- .../druid/testing/guice/DruidTestModule.java | 9 +- .../druid/testing/utils/LoggerListener.java | 6 +- .../org/testng/DruidTestRunnerFactory.java | 47 +- .../tests/indexer/AbstractIndexerTest.java | 36 +- .../io/druid/java/util/common/JodaUtils.java | 12 +- .../util/common/guava/ConcatSequence.java | 11 +- .../util/common/guava/LimitedSequence.java | 7 +- .../java/util/common/GranularityTest.java | 1112 +++++++++++------ .../util/common/guava/SequenceTestHelper.java | 52 +- .../main/java/io/druid/jackson/JodaStuff.java | 46 +- .../last/DoubleLastAggregatorFactory.java | 2 +- .../last/FloatLastAggregatorFactory.java | 2 +- .../extraction/IdentityExtractionFn.java | 4 +- .../epinephelinae/ByteBufferHashTable.java | 2 +- .../epinephelinae/RowBasedGrouperHelper.java | 2 +- .../DoubleGroupByColumnSelectorStrategy.java | 2 +- .../query/lookup/LookupSnapshotTaker.java | 2 +- .../druid/query/topn/PooledTopNAlgorithm.java | 3 +- .../io/druid/query/topn/TopNQueryBuilder.java | 22 +- .../druid/segment/DimensionHandlerUtils.java | 2 + .../druid/segment/StringDimensionIndexer.java | 2 +- .../segment/StringDimensionMergerV9.java | 10 +- .../data/BlockLayoutIndexedFloatSupplier.java | 26 +- .../druid/segment/data/IndexedIterable.java | 4 +- .../io/druid/segment/data/ReadableOffset.java | 4 +- .../segment/serde/ComplexColumnPartSerde.java | 27 +- .../DictionaryEncodedColumnPartSerde.java | 16 +- .../serde/DoubleGenericColumnPartSerde.java | 17 +- .../serde/FloatGenericColumnPartSerde.java | 27 +- .../serde/LongGenericColumnPartSerde.java | 27 +- .../druid/query/MultiValuedDimensionTest.java | 45 +- .../aggregation/FilteredAggregatorTest.java | 4 +- .../GroupByQueryRunnerFactoryTest.java | 11 +- .../query/groupby/GroupByQueryRunnerTest.java | 33 +- .../query/groupby/having/HavingSpecTest.java | 69 +- .../groupby/orderby/TopNSequenceTest.java | 42 +- .../RegisteredLookupExtractionFnTest.java | 59 +- .../TimeBoundaryQueryQueryToolChestTest.java | 2 +- .../timeseries/TimeseriesQueryRunnerTest.java | 16 +- .../druid/query/topn/TopNQueryRunnerTest.java | 10 +- .../druid/query/topn/TopNUnionQueryTest.java | 10 +- .../java/io/druid/segment/IndexIOTest.java | 5 +- .../java/io/druid/segment/MetadataTest.java | 7 +- .../segment/data/IncrementalIndexTest.java | 36 +- .../druid/segment/data/VSizeIndexedTest.java | 2 +- .../OnheapIncrementalIndexBenchmark.java | 11 +- .../druid/guice/FireDepartmentsProvider.java | 6 +- .../loading/LocalDataSegmentPusher.java | 8 +- .../appenderator/AppenderatorPlumber.java | 65 +- .../DefaultRealtimeAppenderatorFactory.java | 3 +- .../firehose/WikipediaIrcDecoder.java | 8 +- .../realtime/plumber/FlushingPlumber.java | 89 +- .../realtime/plumber/RealtimePlumber.java | 65 +- .../server/AsyncQueryForwardingServlet.java | 21 +- .../coordination/broker/DruidBroker.java | 2 +- .../server/coordinator/DruidCoordinator.java | 14 +- .../server/coordinator/LoadQueuePeon.java | 10 +- .../helper/DruidCoordinatorLogger.java | 86 +- .../server/initialization/ZkPathsConfig.java | 27 +- .../client/CachingClusteredClientTest.java | 21 +- .../druid/client/CachingQueryRunnerTest.java | 3 +- .../selector/TierSelectorStrategyTest.java | 17 +- .../io/druid/curator/CuratorTestBase.java | 23 +- .../dimension/LookupDimensionSpecTest.java | 3 +- .../LookupIntrospectionResourceImplTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 60 +- ...DefaultOfflineAppenderatorFactoryTest.java | 73 +- .../firehose/EventReceiverFirehoseTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 63 +- .../BatchDataSegmentAnnouncerTest.java | 4 +- .../coordinator/DruidCoordinatorTest.java | 65 +- .../rules/IntervalLoadRuleTest.java | 103 +- .../server/initialization/JettyTest.java | 16 +- .../shard/SingleDimensionShardSpecTest.java | 15 +- services/src/main/java/io/druid/cli/Main.java | 36 +- .../cli/validate/DruidJsonValidatorTest.java | 18 +- .../io/druid/sql/avatica/DruidStatement.java | 2 +- .../druid/sql/calcite/schema/DruidSchema.java | 11 +- .../sql/avatica/DruidAvaticaHandlerTest.java | 2 +- .../druid/sql/calcite/CalciteQueryTest.java | 38 +- .../druid/sql/calcite/util/CalciteTests.java | 16 +- 142 files changed, 2471 insertions(+), 2359 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FirehoseV2.java b/api/src/main/java/io/druid/data/input/FirehoseV2.java index d22f1a59a4f..026a22461c5 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseV2.java +++ b/api/src/main/java/io/druid/data/input/FirehoseV2.java @@ -22,6 +22,7 @@ package io.druid.data.input; import io.druid.guice.annotations.ExtensionPoint; import java.io.Closeable; + /** * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement @@ -36,56 +37,54 @@ import java.io.Closeable; * 5. GOTO 2 * * Note that commit() is being called *after* advance. - * + * * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this * gets passed around as an Iterator. - *

+ * * The implementation of this interface only needs to be minimally thread-safe. The methods ##start(), ##advance(), * ##currRow() and ##makeCommitter() are all called from the same thread. ##makeCommitter(), however, returns a callback * which will be called on another thread, so the operations inside of that callback must be thread-safe. - *

*/ @ExtensionPoint public interface FirehoseV2 extends Closeable { - /** - * For initial start - * */ - void start() throws Exception; + /** + * For initial start + */ + void start() throws Exception; - /** - * Advance the firehose to the next offset. Implementations of this interface should make sure that - * if advance() is called and throws out an exception, the next call to currRow() should return a - * null value. - * - * @return true if and when there is another row available, false if the stream has dried up - */ - public boolean advance(); + /** + * Advance the firehose to the next offset. Implementations of this interface should make sure that + * if advance() is called and throws out an exception, the next call to currRow() should return a + * null value. + * + * @return true if and when there is another row available, false if the stream has dried up + */ + public boolean advance(); - /** - * @return The current row - */ - public InputRow currRow(); + /** + * @return The current row + */ + public InputRow currRow(); - /** - * Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called. - * - * This method is called when the main processing loop starts to persist its current batch of things to process. - * The returned committer will be run when the current batch has been successfully persisted - * and the metadata the committer carries can also be persisted along with segment data. There is usually - * some time lag between when this method is called and when the runnable is run. The Runnable is also run on - * a separate thread so its operation should be thread-safe. - * - * Note that "correct" usage of this interface will always call advance() before commit() if the current row - * is considered in the commit. - * - * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has - * been committed on the writer side of this interface protocol. - *

- * A simple implementation of this interface might do nothing when run() is called, - * and save proper commit information in metadata - *

- */ - public Committer makeCommitter(); + /** + * Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called. + * + * This method is called when the main processing loop starts to persist its current batch of things to process. + * The returned committer will be run when the current batch has been successfully persisted + * and the metadata the committer carries can also be persisted along with segment data. There is usually + * some time lag between when this method is called and when the runnable is run. The Runnable is also run on + * a separate thread so its operation should be thread-safe. + * + * Note that "correct" usage of this interface will always call advance() before commit() if the current row + * is considered in the commit. + * + * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has + * been committed on the writer side of this interface protocol. + * + * A simple implementation of this interface might do nothing when run() is called, + * and save proper commit information in metadata + */ + public Committer makeCommitter(); } diff --git a/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java b/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java index 883c701bff4..15746a0fb63 100644 --- a/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java +++ b/api/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java @@ -32,7 +32,7 @@ import java.util.List; */ public class CommaListJoinDeserializer extends StdScalarDeserializer> { - protected CommaListJoinDeserializer() + protected CommaListJoinDeserializer() { super(List.class); } diff --git a/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java b/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java index e1f8e01feb7..fd5da6932b4 100644 --- a/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java +++ b/api/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java @@ -25,10 +25,7 @@ public class SingleElementPartitionChunk implements PartitionChunk { private final T element; - public SingleElementPartitionChunk - ( - T element - ) + public SingleElementPartitionChunk(T element) { this.element = element; } diff --git a/api/src/test/java/io/druid/guice/ConditionalMultibindTest.java b/api/src/test/java/io/druid/guice/ConditionalMultibindTest.java index 6b9a23491d8..4cbca482d85 100644 --- a/api/src/test/java/io/druid/guice/ConditionalMultibindTest.java +++ b/api/src/test/java/io/druid/guice/ConditionalMultibindTest.java @@ -268,18 +268,14 @@ public class ConditionalMultibindTest @Override public void configure(Binder binder) { - ConditionalMultibind.create(props, binder, - new TypeLiteral>() - { - } - ).addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set1 - ).addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set2); + ConditionalMultibind + .create(props, binder, new TypeLiteral>() {}) + .addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set1) + .addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set2); - ConditionalMultibind.create(props, binder, - new TypeLiteral>() - { - } - ).addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), zoo1); + ConditionalMultibind + .create(props, binder, new TypeLiteral>() {}) + .addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), zoo1); } }, new Module() @@ -287,46 +283,31 @@ public class ConditionalMultibindTest @Override public void configure(Binder binder) { - ConditionalMultibind.create(props, binder, - new TypeLiteral>() - { - } - ).addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set3); + ConditionalMultibind + .create(props, binder, new TypeLiteral>() {}) + .addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), set3); - ConditionalMultibind.create(props, binder, - new TypeLiteral>() - { - }, - SanDiego.class - ).addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), union); + ConditionalMultibind + .create(props, binder, new TypeLiteral>() {}, SanDiego.class) + .addConditionBinding(ANIMAL_TYPE, Predicates.equalTo("pets"), union); - ConditionalMultibind.create(props, binder, - new TypeLiteral>() - { - } - ).addBinding(new TypeLiteral>() - { - }); + ConditionalMultibind + .create(props, binder, new TypeLiteral>() {}) + .addBinding(new TypeLiteral>() {}); } } ); - Set> actualAnimalSet = injector.getInstance(Key.get(new TypeLiteral>>() - { - })); + Set> actualAnimalSet = injector.getInstance(Key.get(new TypeLiteral>>() {})); Assert.assertEquals(3, actualAnimalSet.size()); Assert.assertEquals(ImmutableSet.of(set1, set2, set3), actualAnimalSet); - actualAnimalSet = injector.getInstance(Key.get(new TypeLiteral>>() - { - }, SanDiego.class)); + actualAnimalSet = injector.getInstance(Key.get(new TypeLiteral>>() {}, SanDiego.class)); Assert.assertEquals(1, actualAnimalSet.size()); Assert.assertEquals(ImmutableSet.of(union), actualAnimalSet); - final Set> actualZooSet = injector.getInstance(Key.get(new TypeLiteral>>() - { - })); + final Set> actualZooSet = injector.getInstance(Key.get(new TypeLiteral>>() {})); Assert.assertEquals(2, actualZooSet.size()); Assert.assertEquals(ImmutableSet.of(zoo1, zoo2), actualZooSet); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java index c5e41f08fd4..bde2eda5ef5 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -179,7 +179,7 @@ public class IndexPersistBenchmark } finally { - FileUtils.deleteDirectory(tmpDir); + FileUtils.deleteDirectory(tmpDir); } } } diff --git a/codestyle/checkstyle-suppressions.xml b/codestyle/checkstyle-suppressions.xml index 1a0071dfb39..1026e6dce88 100644 --- a/codestyle/checkstyle-suppressions.xml +++ b/codestyle/checkstyle-suppressions.xml @@ -48,6 +48,9 @@ + + + diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml index c35fce8b5fe..a376a8141e9 100644 --- a/codestyle/checkstyle.xml +++ b/codestyle/checkstyle.xml @@ -90,6 +90,11 @@ + + + + + diff --git a/common/src/main/java/io/druid/collections/CombiningIterable.java b/common/src/main/java/io/druid/collections/CombiningIterable.java index bcf4a3583ad..67b573f3aeb 100644 --- a/common/src/main/java/io/druid/collections/CombiningIterable.java +++ b/common/src/main/java/io/druid/collections/CombiningIterable.java @@ -19,12 +19,12 @@ package io.druid.collections; -import java.util.Comparator; -import java.util.Iterator; - import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.guava.nary.BinaryFn; +import java.util.Comparator; +import java.util.Iterator; + /** */ public class CombiningIterable implements Iterable @@ -66,9 +66,9 @@ public class CombiningIterable implements Iterable } public static CombiningIterable create( - Iterable it, - Comparator comparator, - BinaryFn fn + Iterable it, + Comparator comparator, + BinaryFn fn ) { return new CombiningIterable(it, comparator, fn); diff --git a/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java b/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java index 5514d9d2a37..50003f8efde 100644 --- a/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java +++ b/common/src/main/java/io/druid/timeline/partition/PartitionHolder.java @@ -19,8 +19,6 @@ package io.druid.timeline.partition; -import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Sets; @@ -121,14 +119,8 @@ public class PartitionHolder implements Iterable> public PartitionChunk getChunk(final int partitionNum) { final Iterator> retVal = Iterators.filter( - holderSet.iterator(), new Predicate>() - { - @Override - public boolean apply(PartitionChunk input) - { - return input.getChunkNumber() == partitionNum; - } - } + holderSet.iterator(), + input -> input.getChunkNumber() == partitionNum ); return retVal.hasNext() ? retVal.next() : null; @@ -142,17 +134,7 @@ public class PartitionHolder implements Iterable> public Iterable payloads() { - return Iterables.transform( - this, - new Function, T>() - { - @Override - public T apply(PartitionChunk input) - { - return input.getObject(); - } - } - ); + return Iterables.transform(this, PartitionChunk::getObject); } @Override diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java index bef5a94ad9f..1838e8952c1 100644 --- a/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java +++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/io/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java @@ -56,15 +56,15 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements private final AmbariMetricsEmitterConfig config; private final String collectorURI; private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = 60000; // default flush wait 1 min - private final ScheduledExecutorService exec = Executors.newScheduledThreadPool(2, new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("AmbariMetricsEmitter-%s") - .build()); // Thread pool of two in order to schedule flush runnable + private final ScheduledExecutorService exec = Executors.newScheduledThreadPool( + 2, // Thread pool of two in order to schedule flush runnable + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("AmbariMetricsEmitter-%s").build() + ); private final AtomicLong countLostEvents = new AtomicLong(0); public AmbariMetricsEmitter( - AmbariMetricsEmitterConfig config, - List emitterList + AmbariMetricsEmitterConfig config, + List emitterList ) { this.config = config; @@ -72,11 +72,11 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements this.timelineMetricConverter = config.getDruidToTimelineEventConverter(); this.eventsQueue = new LinkedBlockingQueue<>(config.getMaxQueueSize()); this.collectorURI = StringUtils.format( - "%s://%s:%s%s", - config.getProtocol(), - config.getHostname(), - config.getPort(), - WS_V1_TIMELINE_METRICS + "%s://%s:%s%s", + config.getProtocol(), + config.getHostname(), + config.getPort(), + WS_V1_TIMELINE_METRICS ); } @@ -90,10 +90,10 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements loadTruststore(config.getTrustStorePath(), config.getTrustStoreType(), config.getTrustStorePassword()); } exec.scheduleAtFixedRate( - new ConsumerRunnable(), - config.getFlushPeriod(), - config.getFlushPeriod(), - TimeUnit.MILLISECONDS + new ConsumerRunnable(), + config.getFlushPeriod(), + config.getFlushPeriod(), + TimeUnit.MILLISECONDS ); started.set(true); } @@ -114,15 +114,15 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements } try { final boolean isSuccessful = eventsQueue.offer( - timelineEvent, - config.getEmitWaitTime(), - TimeUnit.MILLISECONDS + timelineEvent, + config.getEmitWaitTime(), + TimeUnit.MILLISECONDS ); if (!isSuccessful) { if (countLostEvents.getAndIncrement() % 1000 == 0) { log.error( - "Lost total of [%s] events because of emitter queue is full. Please increase the capacity or/and the consumer frequency", - countLostEvents.get() + "Lost total of [%s] events because of emitter queue is full. Please increase the capacity or/and the consumer frequency", + countLostEvents.get() ); } } @@ -163,16 +163,16 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements while (eventsQueue.size() > 0 && !exec.isShutdown()) { try { final TimelineMetric metricEvent = eventsQueue.poll( - config.getWaitForEventTime(), - TimeUnit.MILLISECONDS + config.getWaitForEventTime(), + TimeUnit.MILLISECONDS ); if (metricEvent != null) { metrics.addOrMergeTimelineMetric(metricEvent); if (metrics.getMetrics().size() == batchSize) { emitMetrics(metrics); log.debug( - "sent [%d] events", - metrics.getMetrics().size() + "sent [%d] events", + metrics.getMetrics().size() ); metrics = new TimelineMetrics(); } @@ -186,8 +186,8 @@ public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements if (metrics.getMetrics().size() > 0) { emitMetrics(metrics); log.debug( - "sent [%d] events", - metrics.getMetrics().size() + "sent [%d] events", + metrics.getMetrics().size() ); } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorage.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorage.java index f7e2bbc6440..e42940393ec 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorage.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorage.java @@ -35,7 +35,7 @@ public class GoogleStorage } public void insert(final String bucket, final String path, AbstractInputStreamContent mediaContent) throws IOException - { + { Storage.Objects.Insert insertObject = storage.objects().insert(bucket, null, mediaContent); insertObject.setName(path); insertObject.getMediaHttpUploader().setDirectUploadEnabled(false); diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java index 1af95e016c7..8167e2c879f 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/storage/google/GoogleStorageDruidModule.java @@ -104,7 +104,7 @@ public class GoogleStorageDruidModule implements DruidModule @Provides @LazySingleton public GoogleStorage getGoogleStorage(final GoogleAccountConfig config) - throws IOException, GeneralSecurityException + throws IOException, GeneralSecurityException { LOG.info("Building Cloud Storage Client..."); diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index 0cbac888e92..4d2ae18e466 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -121,12 +121,14 @@ public class OrcIndexGeneratorJobTest private final Interval interval = Intervals.of("2014-10-22T00:00:00Z/P1D"); private File dataRoot; private File outputRoot; - private Integer[][][] shardInfoForEachSegment = new Integer[][][]{{ - {0, 4}, - {1, 4}, - {2, 4}, - {3, 4} - }}; + private Integer[][][] shardInfoForEachSegment = new Integer[][][]{ + { + {0, 4}, + {1, 4}, + {2, 4}, + {3, 4} + } + }; private final InputRowParser inputRowParser = new OrcHadoopInputRowParser( new TimeAndDimsParseSpec( new TimestampSpec("timestamp", "yyyyMMddHH", null), diff --git a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java index e23507d5c1a..11b63a2e0f9 100644 --- a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java +++ b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCache.java @@ -37,154 +37,154 @@ import java.util.concurrent.atomic.AtomicLong; public class RedisCache implements Cache { - private static final Logger log = new Logger(RedisCache.class); + private static final Logger log = new Logger(RedisCache.class); - private JedisPool pool; - private RedisCacheConfig config; + private JedisPool pool; + private RedisCacheConfig config; - private final AtomicLong hitCount = new AtomicLong(0); - private final AtomicLong missCount = new AtomicLong(0); - private final AtomicLong timeoutCount = new AtomicLong(0); - private final AtomicLong errorCount = new AtomicLong(0); + private final AtomicLong hitCount = new AtomicLong(0); + private final AtomicLong missCount = new AtomicLong(0); + private final AtomicLong timeoutCount = new AtomicLong(0); + private final AtomicLong errorCount = new AtomicLong(0); - private final AtomicLong priorRequestCount = new AtomicLong(0); - // both get、put and getBulk will increase request count by 1 - private final AtomicLong totalRequestCount = new AtomicLong(0); + private final AtomicLong priorRequestCount = new AtomicLong(0); + // both get、put and getBulk will increase request count by 1 + private final AtomicLong totalRequestCount = new AtomicLong(0); - private RedisCache(JedisPool pool, RedisCacheConfig config) - { - this.pool = pool; - this.config = config; + private RedisCache(JedisPool pool, RedisCacheConfig config) + { + this.pool = pool; + this.config = config; + } + + public static RedisCache create(final RedisCacheConfig config) + { + JedisPoolConfig poolConfig = new JedisPoolConfig(); + poolConfig.setMaxTotal(config.getMaxTotalConnections()); + poolConfig.setMaxIdle(config.getMaxIdleConnections()); + poolConfig.setMinIdle(config.getMinIdleConnections()); + + JedisPool pool = new JedisPool(poolConfig, config.getHost(), config.getPort(), config.getTimeout()); + return new RedisCache(pool, config); + } + + @Override + public byte[] get(NamedKey key) + { + totalRequestCount.incrementAndGet(); + + try (Jedis jedis = pool.getResource()) { + byte[] bytes = jedis.get(key.toByteArray()); + if (bytes == null) { + missCount.incrementAndGet(); + return null; + } else { + hitCount.incrementAndGet(); + return bytes; + } } - - public static RedisCache create(final RedisCacheConfig config) - { - JedisPoolConfig poolConfig = new JedisPoolConfig(); - poolConfig.setMaxTotal(config.getMaxTotalConnections()); - poolConfig.setMaxIdle(config.getMaxIdleConnections()); - poolConfig.setMinIdle(config.getMinIdleConnections()); - - JedisPool pool = new JedisPool(poolConfig, config.getHost(), config.getPort(), config.getTimeout()); - return new RedisCache(pool, config); + catch (JedisException e) { + if (e.getMessage().contains("Read timed out")) { + timeoutCount.incrementAndGet(); + } else { + errorCount.incrementAndGet(); + } + log.warn(e, "Exception pulling item from cache"); + return null; } + } - @Override - public byte[] get(NamedKey key) - { - totalRequestCount.incrementAndGet(); + @Override + public void put(NamedKey key, byte[] value) + { + totalRequestCount.incrementAndGet(); - try (Jedis jedis = pool.getResource()) { - byte[] bytes = jedis.get(key.toByteArray()); - if (bytes == null) { - missCount.incrementAndGet(); - return null; - } else { - hitCount.incrementAndGet(); - return bytes; - } - } - catch (JedisException e) { - if (e.getMessage().contains("Read timed out")) { - timeoutCount.incrementAndGet(); - } else { - errorCount.incrementAndGet(); - } - log.warn(e, "Exception pulling item from cache"); - return null; + try (Jedis jedis = pool.getResource()) { + jedis.psetex(key.toByteArray(), config.getExpiration(), value); + } + catch (JedisException e) { + errorCount.incrementAndGet(); + log.warn(e, "Exception pushing item to cache"); + } + } + + @Override + public Map getBulk(Iterable keys) + { + totalRequestCount.incrementAndGet(); + + Map results = new HashMap<>(); + + try (Jedis jedis = pool.getResource()) { + List namedKeys = Lists.newArrayList(keys); + List byteKeys = Lists.transform(namedKeys, NamedKey::toByteArray); + + List byteValues = jedis.mget(byteKeys.toArray(new byte[0][])); + + for (int i = 0; i < byteValues.size(); ++i) { + if (byteValues.get(i) != null) { + results.put(namedKeys.get(i), byteValues.get(i)); } + } + + hitCount.addAndGet(results.size()); + missCount.addAndGet(namedKeys.size() - results.size()); + } + catch (JedisException e) { + if (e.getMessage().contains("Read timed out")) { + timeoutCount.incrementAndGet(); + } else { + errorCount.incrementAndGet(); + } + log.warn(e, "Exception pulling items from cache"); } - @Override - public void put(NamedKey key, byte[] value) - { - totalRequestCount.incrementAndGet(); + return results; + } - try (Jedis jedis = pool.getResource()) { - jedis.psetex(key.toByteArray(), config.getExpiration(), value); - } - catch (JedisException e) { - errorCount.incrementAndGet(); - log.warn(e, "Exception pushing item to cache"); - } + @Override + public void close(String namespace) + { + // no resources to cleanup + } + + @Override + public CacheStats getStats() + { + return new CacheStats( + hitCount.get(), + missCount.get(), + 0, + 0, + 0, + timeoutCount.get(), + errorCount.get() + ); + } + + @Override + public boolean isLocal() + { + return false; + } + + @Override + public void doMonitor(ServiceEmitter emitter) + { + final long priorCount = priorRequestCount.get(); + final long totalCount = totalRequestCount.get(); + final ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder(); + emitter.emit(builder.build("query/cache/redis/total/requests", totalCount)); + emitter.emit(builder.build("query/cache/redis/delta/requests", totalCount - priorCount)); + if (!priorRequestCount.compareAndSet(priorCount, totalCount)) { + log.error("Prior value changed while I was reporting! updating anyways"); + priorRequestCount.set(totalCount); } + } - @Override - public Map getBulk(Iterable keys) - { - totalRequestCount.incrementAndGet(); - - Map results = new HashMap<>(); - - try (Jedis jedis = pool.getResource()) { - List namedKeys = Lists.newArrayList(keys); - List byteKeys = Lists.transform(namedKeys, NamedKey::toByteArray); - - List byteValues = jedis.mget(byteKeys.toArray(new byte[0][])); - - for (int i = 0; i < byteValues.size(); ++i) { - if (byteValues.get(i) != null) { - results.put(namedKeys.get(i), byteValues.get(i)); - } - } - - hitCount.addAndGet(results.size()); - missCount.addAndGet(namedKeys.size() - results.size()); - } - catch (JedisException e) { - if (e.getMessage().contains("Read timed out")) { - timeoutCount.incrementAndGet(); - } else { - errorCount.incrementAndGet(); - } - log.warn(e, "Exception pulling items from cache"); - } - - return results; - } - - @Override - public void close(String namespace) - { - // no resources to cleanup - } - - @Override - public CacheStats getStats() - { - return new CacheStats( - hitCount.get(), - missCount.get(), - 0, - 0, - 0, - timeoutCount.get(), - errorCount.get() - ); - } - - @Override - public boolean isLocal() - { - return false; - } - - @Override - public void doMonitor(ServiceEmitter emitter) - { - final long priorCount = priorRequestCount.get(); - final long totalCount = totalRequestCount.get(); - final ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder(); - emitter.emit(builder.build("query/cache/redis/total/requests", totalCount)); - emitter.emit(builder.build("query/cache/redis/delta/requests", totalCount - priorCount)); - if (!priorRequestCount.compareAndSet(priorCount, totalCount)) { - log.error("Prior value changed while I was reporting! updating anyways"); - priorRequestCount.set(totalCount); - } - } - - @VisibleForTesting - static RedisCache create(final JedisPool pool, final RedisCacheConfig config) - { - return new RedisCache(pool, config); - } + @VisibleForTesting + static RedisCache create(final JedisPool pool, final RedisCacheConfig config) + { + return new RedisCache(pool, config); + } } diff --git a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheConfig.java b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheConfig.java index 14c2d472e2b..3b61814ea9d 100644 --- a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheConfig.java +++ b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheConfig.java @@ -23,64 +23,64 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class RedisCacheConfig { - @JsonProperty - private String host; + @JsonProperty + private String host; - @JsonProperty - private int port; + @JsonProperty + private int port; - // milliseconds, default to one day - @JsonProperty - private long expiration = 24 * 3600 * 1000; + // milliseconds, default to one day + @JsonProperty + private long expiration = 24 * 3600 * 1000; - // milliseconds, the type is 'int' because current Jedis only accept 'int' for timeout - @JsonProperty - private int timeout = 2000; + // milliseconds, the type is 'int' because current Jedis only accept 'int' for timeout + @JsonProperty + private int timeout = 2000; - // max connections of redis connection pool - @JsonProperty - private int maxTotalConnections = 8; + // max connections of redis connection pool + @JsonProperty + private int maxTotalConnections = 8; - // max idle connections of redis connection pool - @JsonProperty - private int maxIdleConnections = 8; + // max idle connections of redis connection pool + @JsonProperty + private int maxIdleConnections = 8; - // min idle connections of redis connection pool - @JsonProperty - private int minIdleConnections = 0; + // min idle connections of redis connection pool + @JsonProperty + private int minIdleConnections = 0; - public String getHost() - { - return host; - } + public String getHost() + { + return host; + } - public int getPort() - { - return port; - } + public int getPort() + { + return port; + } - public long getExpiration() - { - return expiration; - } + public long getExpiration() + { + return expiration; + } - public int getTimeout() - { - return timeout; - } + public int getTimeout() + { + return timeout; + } - public int getMaxTotalConnections() - { - return maxTotalConnections; - } + public int getMaxTotalConnections() + { + return maxTotalConnections; + } - public int getMaxIdleConnections() - { - return maxIdleConnections; - } + public int getMaxIdleConnections() + { + return maxIdleConnections; + } - public int getMinIdleConnections() - { - return minIdleConnections; - } + public int getMinIdleConnections() + { + return minIdleConnections; + } } diff --git a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheProvider.java b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheProvider.java index 4b743309680..bdd7d018450 100644 --- a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheProvider.java +++ b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisCacheProvider.java @@ -24,9 +24,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; @JsonTypeName("redis") public class RedisCacheProvider extends RedisCacheConfig implements CacheProvider { - @Override - public Cache get() - { - return RedisCache.create(this); - } + @Override + public Cache get() + { + return RedisCache.create(this); + } } diff --git a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisDruidModule.java b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisDruidModule.java index c4814b2448d..c6acc618a84 100644 --- a/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisDruidModule.java +++ b/extensions-contrib/redis-cache/src/main/java/io/druid/client/cache/RedisDruidModule.java @@ -29,18 +29,15 @@ import java.util.List; public class RedisDruidModule implements DruidModule { - @Override - public void configure(Binder binder) - { + @Override + public void configure(Binder binder) + { + // do nothing + } - } - - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule("DruidRedisCache") - .registerSubtypes(RedisCacheProvider.class) - ); - } + @Override + public List getJacksonModules() + { + return ImmutableList.of(new SimpleModule("DruidRedisCache").registerSubtypes(RedisCacheProvider.class)); + } } diff --git a/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java b/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java index a0219bcdc7b..2817bb013aa 100644 --- a/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java +++ b/extensions-contrib/redis-cache/src/test/java/io/druid/client/cache/RedisCacheTest.java @@ -43,169 +43,170 @@ import java.util.UUID; public class RedisCacheTest { - private static final byte[] HI = StringUtils.toUtf8("hiiiiiiiiiiiiiiiiiii"); - private static final byte[] HO = StringUtils.toUtf8("hooooooooooooooooooo"); + private static final byte[] HI = StringUtils.toUtf8("hiiiiiiiiiiiiiiiiiii"); + private static final byte[] HO = StringUtils.toUtf8("hooooooooooooooooooo"); - private RedisCache cache; - private final RedisCacheConfig cacheConfig = new RedisCacheConfig() + private RedisCache cache; + private final RedisCacheConfig cacheConfig = new RedisCacheConfig() + { + @Override + public int getTimeout() { - @Override - public int getTimeout() - { - return 10; - } + return 10; + } - @Override - public long getExpiration() - { - return 3600000; - } - }; - - @Before - public void setUp() throws Exception + @Override + public long getExpiration() { - JedisPoolConfig poolConfig = new JedisPoolConfig(); - poolConfig.setMaxTotal(cacheConfig.getMaxTotalConnections()); - poolConfig.setMaxIdle(cacheConfig.getMaxIdleConnections()); - poolConfig.setMinIdle(cacheConfig.getMinIdleConnections()); + return 3600000; + } + }; - MockJedisPool pool = new MockJedisPool(poolConfig, "localhost"); - // orginal MockJedis do not support 'milliseconds' in long type, - // for test we override to support it - pool.setClient(new MockJedis("localhost") { - @Override - public String psetex(byte[] key, long milliseconds, byte[] value) - { - return this.psetex(key, (int) milliseconds, value); + @Before + public void setUp() throws Exception + { + JedisPoolConfig poolConfig = new JedisPoolConfig(); + poolConfig.setMaxTotal(cacheConfig.getMaxTotalConnections()); + poolConfig.setMaxIdle(cacheConfig.getMaxIdleConnections()); + poolConfig.setMinIdle(cacheConfig.getMinIdleConnections()); + + MockJedisPool pool = new MockJedisPool(poolConfig, "localhost"); + // orginal MockJedis do not support 'milliseconds' in long type, + // for test we override to support it + pool.setClient(new MockJedis("localhost") + { + @Override + public String psetex(byte[] key, long milliseconds, byte[] value) + { + return this.psetex(key, (int) milliseconds, value); + } + }); + + cache = RedisCache.create(pool, cacheConfig); + } + + @Test + public void testBasicInjection() throws Exception + { + final RedisCacheConfig config = new RedisCacheConfig(); + Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test/redis"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + + binder.bind(RedisCacheConfig.class).toInstance(config); + binder.bind(Cache.class).toProvider(RedisCacheProviderWithConfig.class).in(ManageLifecycle.class); } - }); - - cache = RedisCache.create(pool, cacheConfig); + ) + ); + Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + lifecycle.start(); + try { + Cache cache = injector.getInstance(Cache.class); + Assert.assertEquals(RedisCache.class, cache.getClass()); } - - @Test - public void testBasicInjection() throws Exception - { - final RedisCacheConfig config = new RedisCacheConfig(); - Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), ImmutableList.of( - binder -> { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test/redis"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - - binder.bind(RedisCacheConfig.class).toInstance(config); - binder.bind(Cache.class).toProvider(RedisCacheProviderWithConfig.class).in(ManageLifecycle.class); - } - ) - ); - Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - lifecycle.start(); - try { - Cache cache = injector.getInstance(Cache.class); - Assert.assertEquals(RedisCache.class, cache.getClass()); - } - finally { - lifecycle.stop(); - } + finally { + lifecycle.stop(); } + } - @Test - public void testSimpleInjection() - { - final String uuid = UUID.randomUUID().toString(); - System.setProperty(uuid + ".type", "redis"); - final Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), ImmutableList.of( - binder -> { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test/redis"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + @Test + public void testSimpleInjection() + { + final String uuid = UUID.randomUUID().toString(); + System.setProperty(uuid + ".type", "redis"); + final Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/test/redis"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - binder.bind(Cache.class).toProvider(CacheProvider.class); - JsonConfigProvider.bind(binder, uuid, CacheProvider.class); - } - ) - ); - final CacheProvider cacheProvider = injector.getInstance(CacheProvider.class); - Assert.assertNotNull(cacheProvider); - Assert.assertEquals(RedisCacheProvider.class, cacheProvider.getClass()); - } + binder.bind(Cache.class).toProvider(CacheProvider.class); + JsonConfigProvider.bind(binder, uuid, CacheProvider.class); + } + ) + ); + final CacheProvider cacheProvider = injector.getInstance(CacheProvider.class); + Assert.assertNotNull(cacheProvider); + Assert.assertEquals(RedisCacheProvider.class, cacheProvider.getClass()); + } - @Test - public void testSanity() throws Exception - { - Assert.assertNull(cache.get(new Cache.NamedKey("a", HI))); - put(cache, "a", HI, 0); - Assert.assertEquals(0, get(cache, "a", HI)); - Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); + @Test + public void testSanity() throws Exception + { + Assert.assertNull(cache.get(new Cache.NamedKey("a", HI))); + put(cache, "a", HI, 0); + Assert.assertEquals(0, get(cache, "a", HI)); + Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); - put(cache, "the", HI, 1); - Assert.assertEquals(0, get(cache, "a", HI)); - Assert.assertEquals(1, get(cache, "the", HI)); + put(cache, "the", HI, 1); + Assert.assertEquals(0, get(cache, "a", HI)); + Assert.assertEquals(1, get(cache, "the", HI)); - put(cache, "the", HO, 10); - Assert.assertEquals(0, get(cache, "a", HI)); - Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); - Assert.assertEquals(1, get(cache, "the", HI)); - Assert.assertEquals(10, get(cache, "the", HO)); + put(cache, "the", HO, 10); + Assert.assertEquals(0, get(cache, "a", HI)); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); + Assert.assertEquals(1, get(cache, "the", HI)); + Assert.assertEquals(10, get(cache, "the", HO)); - cache.close("the"); - Assert.assertEquals(0, get(cache, "a", HI)); - Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); - } + cache.close("the"); + Assert.assertEquals(0, get(cache, "a", HI)); + Assert.assertNull(cache.get(new Cache.NamedKey("a", HO))); + } - @Test - public void testGetBulk() throws Exception - { - Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); + @Test + public void testGetBulk() throws Exception + { + Assert.assertNull(cache.get(new Cache.NamedKey("the", HI))); - put(cache, "the", HI, 1); - put(cache, "the", HO, 10); + put(cache, "the", HI, 1); + put(cache, "the", HO, 10); - Cache.NamedKey key1 = new Cache.NamedKey("the", HI); - Cache.NamedKey key2 = new Cache.NamedKey("the", HO); - Cache.NamedKey key3 = new Cache.NamedKey("a", HI); + Cache.NamedKey key1 = new Cache.NamedKey("the", HI); + Cache.NamedKey key2 = new Cache.NamedKey("the", HO); + Cache.NamedKey key3 = new Cache.NamedKey("a", HI); - Map result = cache.getBulk( - Lists.newArrayList( - key1, - key2, - key3 - ) - ); + Map result = cache.getBulk( + Lists.newArrayList( + key1, + key2, + key3 + ) + ); - Assert.assertEquals(1, Ints.fromByteArray(result.get(key1))); - Assert.assertEquals(10, Ints.fromByteArray(result.get(key2))); - Assert.assertEquals(null, result.get(key3)); - } + Assert.assertEquals(1, Ints.fromByteArray(result.get(key1))); + Assert.assertEquals(10, Ints.fromByteArray(result.get(key2))); + Assert.assertEquals(null, result.get(key3)); + } - public void put(Cache cache, String namespace, byte[] key, Integer value) - { - cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value)); - } + public void put(Cache cache, String namespace, byte[] key, Integer value) + { + cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value)); + } - public int get(Cache cache, String namespace, byte[] key) - { - return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key))); - } + public int get(Cache cache, String namespace, byte[] key) + { + return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key))); + } } class RedisCacheProviderWithConfig extends RedisCacheProvider { - private final RedisCacheConfig config; + private final RedisCacheConfig config; - @Inject - public RedisCacheProviderWithConfig(RedisCacheConfig config) - { - this.config = config; - } + @Inject + public RedisCacheProviderWithConfig(RedisCacheConfig config) + { + this.config = config; + } - @Override - public Cache get() - { - return RedisCache.create(config); - } + @Override + public Cache get() + { + return RedisCache.create(config); + } } diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java index d1b3200dd22..30df67140a8 100644 --- a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitter.java @@ -42,34 +42,33 @@ public class StatsDEmitter implements Emitter private final static String DRUID_METRIC_SEPARATOR = "\\/"; private final static String STATSD_SEPARATOR = ":|\\|"; + static final StatsDEmitter of(StatsDEmitterConfig config, ObjectMapper mapper) + { + NonBlockingStatsDClient client = new NonBlockingStatsDClient( + config.getPrefix(), + config.getHostname(), + config.getPort(), + new StatsDClientErrorHandler() + { + private int exceptionCount = 0; + + @Override + public void handle(Exception exception) + { + if (exceptionCount % 1000 == 0) { + log.error(exception, "Error sending metric to StatsD."); + } + exceptionCount += 1; + } + } + ); + return new StatsDEmitter(config, mapper, client); + } + private final StatsDClient statsd; private final StatsDEmitterConfig config; private final DimensionConverter converter; - public StatsDEmitter(StatsDEmitterConfig config, ObjectMapper mapper) - { - this(config, mapper, - new NonBlockingStatsDClient( - config.getPrefix(), - config.getHostname(), - config.getPort(), - new StatsDClientErrorHandler() - { - private int exceptionCount = 0; - - @Override - public void handle(Exception exception) - { - if (exceptionCount % 1000 == 0) { - log.error(exception, "Error sending metric to StatsD."); - } - exceptionCount += 1; - } - } - ) - ); - } - public StatsDEmitter(StatsDEmitterConfig config, ObjectMapper mapper, StatsDClient client) { this.config = config; diff --git a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java index ad0b8f8cd75..3d00da34eff 100644 --- a/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java +++ b/extensions-contrib/statsd-emitter/src/main/java/io/druid/emitter/statsd/StatsDEmitterModule.java @@ -55,6 +55,6 @@ public class StatsDEmitterModule implements DruidModule @Named(EMITTER_TYPE) public Emitter getEmitter(StatsDEmitterConfig config, ObjectMapper mapper) { - return new StatsDEmitter(config, mapper); + return StatsDEmitter.of(config, mapper); } } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java index 979a10fcab5..51b1d366b60 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java @@ -62,7 +62,9 @@ public class KerberosHttpClient extends AbstractHttpClient @Override public ListenableFuture go( - Request request, HttpResponseHandler httpResponseHandler, Duration duration + Request request, + HttpResponseHandler httpResponseHandler, + Duration duration ) { final SettableFuture retVal = SettableFuture.create(); @@ -72,10 +74,10 @@ public class KerberosHttpClient extends AbstractHttpClient private void inner_go( - final Request request, - final HttpResponseHandler httpResponseHandler, - final Duration duration, - final SettableFuture future + final Request request, + final HttpResponseHandler httpResponseHandler, + final Duration duration, + final SettableFuture future ) { try { @@ -92,9 +94,9 @@ public class KerberosHttpClient extends AbstractHttpClient if (DruidKerberosUtil.needToSendCredentials(cookieManager.getCookieStore(), uri)) { // No Cookies for requested URI, authenticate user and add authentication header log.debug( - "No Auth Cookie found for URI[%s]. Existing Cookies[%s] Authenticating... ", - uri, - cookieManager.getCookieStore().getCookies() + "No Auth Cookie found for URI[%s]. Existing Cookies[%s] Authenticating... ", + uri, + cookieManager.getCookieStore().getCookies() ); DruidKerberosUtil.authenticateIfRequired(internalClientPrincipal, internalClientKeytab); UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); @@ -114,13 +116,11 @@ public class KerberosHttpClient extends AbstractHttpClient } ListenableFuture> internalFuture = delegate.go( - request, - new RetryIfUnauthorizedResponseHandler(new ResponseCookieHandler( - request.getUrl().toURI(), - cookieManager, - httpResponseHandler - )), - duration + request, + new RetryIfUnauthorizedResponseHandler( + new ResponseCookieHandler(request.getUrl().toURI(), cookieManager, httpResponseHandler) + ), + duration ); Futures.addCallback(internalFuture, new FutureCallback>() diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java index 4e2bed41904..5a4d4df0b43 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/ResponseCookieHandler.java @@ -71,9 +71,7 @@ public class ResponseCookieHandler implements HttpResponseH } @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk httpChunk - ) + public ClientResponse handleChunk(ClientResponse clientResponse, HttpChunk httpChunk) { return delegate.handleChunk(clientResponse, httpChunk); } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java index a2f960d9b5d..8d8148842c8 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/RetryIfUnauthorizedResponseHandler.java @@ -27,15 +27,13 @@ import org.jboss.netty.handler.codec.http.HttpResponse; import org.jboss.netty.handler.codec.http.HttpResponseStatus; public class RetryIfUnauthorizedResponseHandler - implements HttpResponseHandler, RetryResponseHolder> + implements HttpResponseHandler, RetryResponseHolder> { private static final Logger log = new Logger(RetryIfUnauthorizedResponseHandler.class); private final HttpResponseHandler httpResponseHandler; - public RetryIfUnauthorizedResponseHandler( - HttpResponseHandler httpResponseHandler - ) + public RetryIfUnauthorizedResponseHandler(HttpResponseHandler httpResponseHandler) { this.httpResponseHandler = httpResponseHandler; } @@ -55,7 +53,8 @@ public class RetryIfUnauthorizedResponseHandler @Override public ClientResponse> handleChunk( - ClientResponse> clientResponse, HttpChunk httpChunk + ClientResponse> clientResponse, + HttpChunk httpChunk ) { if (clientResponse.getObj().shouldRetry()) { diff --git a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java index 1c63d09f1da..e1ee9860632 100644 --- a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java +++ b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/SpnegoFilterConfigTest.java @@ -43,24 +43,24 @@ public class SpnegoFilterConfigTest public void testserde() { Injector injector = Guice.createInjector( - new Module() - { - @Override - public void configure(Binder binder) + new Module() { - binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties"))); - binder.install(new ConfigModule()); - binder.install(new DruidGuiceExtensions()); - JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); - } + @Override + public void configure(Binder binder) + { + binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties"))); + binder.install(new ConfigModule()); + binder.install(new DruidGuiceExtensions()); + JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); + } - @Provides - @LazySingleton - public ObjectMapper jsonMapper() - { - return new DefaultObjectMapper(); + @Provides + @LazySingleton + public ObjectMapper jsonMapper() + { + return new DefaultObjectMapper(); + } } - } ); Properties props = injector.getInstance(Properties.class); diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java index f5baebb9339..cad4b9b2f1b 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/storage/hdfs/HdfsDataSegmentPusherTest.java @@ -382,23 +382,23 @@ public class HdfsDataSegmentPusherTest try { schema = objectMapper.readValue( "{\n" - + " \"dataSchema\": {\n" - + " \"dataSource\": \"source\",\n" - + " \"metricsSpec\": [],\n" - + " \"granularitySpec\": {\n" - + " \"type\": \"uniform\",\n" - + " \"segmentGranularity\": \"hour\",\n" - + " \"intervals\": [\"2012-07-10/P1D\"]\n" - + " }\n" - + " },\n" - + " \"ioConfig\": {\n" - + " \"type\": \"hadoop\",\n" - + " \"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"\n" - + " }\n" - + "}", - HadoopIngestionSpec.class - ); - } + + " \"dataSchema\": {\n" + + " \"dataSource\": \"source\",\n" + + " \"metricsSpec\": [],\n" + + " \"granularitySpec\": {\n" + + " \"type\": \"uniform\",\n" + + " \"segmentGranularity\": \"hour\",\n" + + " \"intervals\": [\"2012-07-10/P1D\"]\n" + + " }\n" + + " },\n" + + " \"ioConfig\": {\n" + + " \"type\": \"hadoop\",\n" + + " \"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"\n" + + " }\n" + + "}", + HadoopIngestionSpec.class + ); + } catch (Exception e) { throw Throwables.propagate(e); } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java index 61845127213..f1907fabeeb 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -1183,11 +1183,7 @@ public class ApproximateHistogram public boolean canStoreCompact() { final long exactCount = getExactCount(); - return ( - size <= Short.MAX_VALUE - && exactCount <= Byte.MAX_VALUE - && (count - exactCount) <= Byte.MAX_VALUE - ); + return (size <= Short.MAX_VALUE && exactCount <= Byte.MAX_VALUE && (count - exactCount) <= Byte.MAX_VALUE); } /** diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index b48ea4b6708..d06d673b394 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -19,7 +19,6 @@ package io.druid.query.aggregation.histogram; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -75,14 +74,7 @@ public class ApproximateHistogramTopNQueryTest new TopNQueryRunnerFactory( new StupidPool( "TopNQueryRunnerFactory-bufferPool", - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(2000); - } - } + () -> ByteBuffer.allocate(2000) ), new TopNQueryQueryToolChest( new TopNQueryConfig(), diff --git a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java index 8e1d8bd38c9..4aa418663d6 100644 --- a/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java +++ b/extensions-core/kafka-extraction-namespace/src/main/java/io/druid/query/lookup/KafkaLookupExtractorFactory.java @@ -343,8 +343,7 @@ public class KafkaLookupExtractorFactory implements LookupExtractorFactory return !(getKafkaTopic().equals(that.getKafkaTopic()) && getKafkaProperties().equals(that.getKafkaProperties()) && getConnectTimeout() == that.getConnectTimeout() - && isInjective() == that.isInjective() - ); + && isInjective() == that.isInjective()); } @Nullable diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 453405ce08e..7035a8d704b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1496,22 +1496,25 @@ public class KafkaIndexTaskTest private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() { + IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( + null, + null, + null + ) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + }; return new DefaultQueryRunnerFactoryConglomerate( ImmutableMap., QueryRunnerFactory>of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate( - QueryRunner delegate, QueryToolChest> toolChest - ) - { - return delegate; - } - } - ), + new TimeseriesQueryQueryToolChest(queryRunnerDecorator), new TimeseriesQueryEngine(), new QueryWatcher() { @@ -1613,6 +1616,14 @@ public class KafkaIndexTaskTest final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; toolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, @@ -1628,17 +1639,7 @@ public class KafkaIndexTaskTest MoreExecutors.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( - new SegmentLoaderLocalCacheManager( - null, - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Lists.newArrayList(); - } - }, testUtils.getTestObjectMapper() - ) + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) ), testUtils.getTestObjectMapper(), testUtils.getTestIndexIO(), diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java index 80adc7a3e91..38078fb19da 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/query/lookup/NamespaceLookupExtractorFactoryTest.java @@ -406,7 +406,9 @@ public class NamespaceLookupExtractorFactoryTest final NamespaceLookupExtractorFactory f1 = new NamespaceLookupExtractorFactory( en1, scheduler - ), f2 = new NamespaceLookupExtractorFactory(en2, scheduler), f1b = new NamespaceLookupExtractorFactory( + ); + final NamespaceLookupExtractorFactory f2 = new NamespaceLookupExtractorFactory(en2, scheduler); + final NamespaceLookupExtractorFactory f1b = new NamespaceLookupExtractorFactory( en1, scheduler ); diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java index e983c77b5b7..0d0ece41e53 100644 --- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java @@ -269,17 +269,12 @@ public class JdbcExtractionNamespaceTest } ); - try (final Closeable closeable = - new Closeable() - { - @Override - public void close() throws IOException - { - if (!setupFuture.isDone() && !setupFuture.cancel(true) && !setupFuture.isDone()) { - throw new IOException("Unable to stop future"); - } - } - }) { + Closeable closeable = () -> { + if (!setupFuture.isDone() && !setupFuture.cancel(true) && !setupFuture.isDone()) { + throw new IOException("Unable to stop future"); + } + }; + try (final Closeable c = closeable) { handleRef = setupFuture.get(10, TimeUnit.SECONDS); } Assert.assertNotNull(handleRef); diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LookupExtractionModule.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LookupExtractionModule.java index bb5873b7001..4fbc4268f33 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LookupExtractionModule.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/LookupExtractionModule.java @@ -55,6 +55,6 @@ public class LookupExtractionModule implements DruidModule public static byte[] getRandomCacheKey() { - return StringUtils.toUtf8(UUID.randomUUID().toString()); + return StringUtils.toUtf8(UUID.randomUUID().toString()); } } diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/OffHeapLoadingCache.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/OffHeapLoadingCache.java index c9cdecc86ee..9b1fe44583a 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/OffHeapLoadingCache.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/cache/loading/OffHeapLoadingCache.java @@ -124,8 +124,7 @@ public class OffHeapLoadingCache implements LoadingCache public Map getAllPresent(final Iterable keys) { ImmutableMap.Builder builder = ImmutableMap.builder(); - for (K key : keys - ) { + for (K key : keys) { V value = getIfPresent(key); if (value != null) { builder.put(key, value); diff --git a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/jdbc/JdbcDataFetcher.java b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/jdbc/JdbcDataFetcher.java index bb133b7afac..6d0af4e821f 100644 --- a/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/jdbc/JdbcDataFetcher.java +++ b/extensions-core/lookups-cached-single/src/main/java/io/druid/server/lookup/jdbc/JdbcDataFetcher.java @@ -105,22 +105,12 @@ public class JdbcDataFetcher implements DataFetcher @Override public Iterable> fetchAll() { - return inReadOnlyTransaction(new TransactionCallback>>() - { - @Override - public List> inTransaction( - Handle handle, - TransactionStatus status - ) throws Exception - { - return handle.createQuery(fetchAllQuery) - .setFetchSize(streamingFetchSize) - .map(new KeyValueResultSetMapper(keyColumn, valueColumn)) - .list(); - } - - } - ); + return inReadOnlyTransaction((handle, status) -> { + return handle.createQuery(fetchAllQuery) + .setFetchSize(streamingFetchSize) + .map(new KeyValueResultSetMapper(keyColumn, valueColumn)) + .list(); + }); } @Override diff --git a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java index b15de57b7ed..c186b8973e4 100644 --- a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/cache/loading/LoadingCacheTest.java @@ -46,14 +46,8 @@ public class LoadingCacheTest public static Collection inputData() { return Arrays.asList(new Object[][]{ - {new OnHeapLoadingCache<>(4, 1000, null, null, null)}, { - new OffHeapLoadingCache( - 0, - 0L, - 0L, - 0L - ) - } + {new OnHeapLoadingCache<>(4, 1000, null, null, null)}, + {new OffHeapLoadingCache(0, 0L, 0L, 0L)} }); } diff --git a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/jdbc/JdbcDataFetcherTest.java b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/jdbc/JdbcDataFetcherTest.java index 41c29b0b04c..21c3ed3c5ca 100644 --- a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/jdbc/JdbcDataFetcherTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/jdbc/JdbcDataFetcherTest.java @@ -112,8 +112,7 @@ public class JdbcDataFetcherTest public void testFetchKeys() { ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); - for (Map.Entry entry: jdbcDataFetcher.fetch(lookupMap.keySet()) - ) { + for (Map.Entry entry: jdbcDataFetcher.fetch(lookupMap.keySet())) { mapBuilder.put(entry.getKey(), entry.getValue()); } diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java index 58e0b79ac9b..3c5dfbca318 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/io/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -92,10 +92,8 @@ public class PostgreSQLConnector extends SQLMetadataConnector { if (canUpsert == null) { DatabaseMetaData metaData = handle.getConnection().getMetaData(); - canUpsert = metaData.getDatabaseMajorVersion() > 9 || ( - metaData.getDatabaseMajorVersion() == 9 && - metaData.getDatabaseMinorVersion() >= 5 - ); + canUpsert = metaData.getDatabaseMajorVersion() > 9 || + (metaData.getDatabaseMajorVersion() == 9 && metaData.getDatabaseMinorVersion() >= 5); } return canUpsert; } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java index 5d0e4bd4b00..67af162759b 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentFinderTest.java @@ -354,8 +354,7 @@ public class S3DataSegmentFinderTest }; ImmutableList keys = ImmutableList.copyOf( - Ordering.natural().sortedCopy(Iterables.filter(keysOrigin, prefixFilter) - ) + Ordering.natural().sortedCopy(Iterables.filter(keysOrigin, prefixFilter)) ); int startOffset = 0; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index 820e5034345..6f88a0c3d09 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -137,9 +137,8 @@ public class DetermineHashedPartitionsJob implements Jobby throw new ISE("Path[%s] didn't exist!?", intervalInfoPath); } List intervals = config.JSON_MAPPER.readValue( - Utils.openInputStream(groupByJob, intervalInfoPath), new TypeReference>() - { - } + Utils.openInputStream(groupByJob, intervalInfoPath), + new TypeReference>() {} ); config.setGranularitySpec( new UniformGranularitySpec( @@ -162,9 +161,8 @@ public class DetermineHashedPartitionsJob implements Jobby } if (Utils.exists(groupByJob, fileSystem, partitionInfoPath)) { final Long numRows = config.JSON_MAPPER.readValue( - Utils.openInputStream(groupByJob, partitionInfoPath), new TypeReference() - { - } + Utils.openInputStream(groupByJob, partitionInfoPath), + new TypeReference() {} ); log.info("Found approximately [%,d] rows in data.", numRows); @@ -272,11 +270,9 @@ public class DetermineHashedPartitionsJob implements Jobby } interval = maybeInterval.get(); } - hyperLogLogs.get(interval) - .add( - hashFunction.hashBytes(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)) - .asBytes() - ); + hyperLogLogs + .get(interval) + .add(hashFunction.hashBytes(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsBytes(groupKey)).asBytes()); } @Override diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index 4ab2e6bba37..5f3c15781c8 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -54,9 +54,8 @@ import java.util.Map; public class JobHelperTest { - public final @Rule - TemporaryFolder temporaryFolder = new TemporaryFolder(); + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private HadoopDruidIndexerConfig config; private File tmpDir; diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpecTest.java index fd0a6aed948..d726e5cfd87 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/MetadataStorageUpdaterJobSpecTest.java @@ -36,7 +36,8 @@ public class MetadataStorageUpdaterJobSpecTest "jdbc:mysql://localhost/druid", "druid", "\"nothing\"", - "nothing"); + "nothing" + ); } @Test @@ -48,17 +49,18 @@ public class MetadataStorageUpdaterJobSpecTest "jdbc:mysql://localhost/druid", "druid", "{\"type\":\"default\",\"password\":\"nothing\"}", - "nothing"); + "nothing" + ); } private void testMetadataStorageUpdaterJobSpec( - String segmentTable, - String type, - String connectURI, - String user, - String pwdString, - String pwd - ) throws Exception + String segmentTable, + String type, + String connectURI, + String user, + String pwdString, + String pwd + ) throws Exception { MetadataStorageUpdaterJobSpec spec = jsonMapper.readValue( "{" + @@ -68,7 +70,8 @@ public class MetadataStorageUpdaterJobSpecTest "\"password\": " + pwdString + ",\n" + "\"segmentTable\": \"" + segmentTable + "\"\n" + "}", - MetadataStorageUpdaterJobSpec.class); + MetadataStorageUpdaterJobSpec.class + ); Assert.assertEquals(segmentTable, spec.getSegmentTable()); Assert.assertEquals(type, spec.getType()); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java index d8c1a3f7dfb..f9366fd4bff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java @@ -171,7 +171,7 @@ public class MetadataTaskStorage implements TaskStorage @Override public Optional getTask(final String taskId) { - return handler.getEntry(taskId); + return handler.getEntry(taskId); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 522ffa4ab40..422cf84e9a2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -515,16 +515,7 @@ public class TaskLockbox } return ImmutableList.copyOf( - Iterables.filter( - searchSpace, new Predicate() - { - @Override - public boolean apply(TaskLockPosse taskLock) - { - return taskLock.getTaskIds().contains(task.getId()); - } - } - ) + Iterables.filter(searchSpace, taskLock -> taskLock.getTaskIds().contains(task.getId())) ); } finally { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index 989b97b0a2a..4fa12b6decf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -77,7 +77,7 @@ public class TaskMaster final SupervisorManager supervisorManager, final OverlordHelperManager overlordHelperManager, @IndexingService final DruidLeaderSelector overlordLeaderSelector - ) + ) { this.supervisorManager = supervisorManager; this.taskActionClientFactory = taskActionClientFactory; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleConfig.java index 5c81b848024..44467ea6559 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -94,10 +94,10 @@ public class ExecutorLifecycleConfig public InputStream getParentStream() { - if ("stdin".equals(parentStreamName)) { - return System.in; - } else { - throw new ISE("Unknown stream name[%s]", parentStreamName); - } + if ("stdin".equals(parentStreamName)) { + return System.in; + } else { + throw new ISE("Unknown stream name[%s]", parentStreamName); + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java index 77a0f0fa93e..f9cb242218a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java @@ -29,12 +29,8 @@ import io.druid.indexing.common.task.TaskResource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartment; -import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.segment.realtime.plumber.Plumber; -import io.druid.segment.realtime.plumber.PlumberSchool; import java.io.File; @@ -58,18 +54,13 @@ public class TestRealtimeTask extends RealtimeIndexTask id, taskResource, new FireDepartment( - new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, mapper), new RealtimeIOConfig( - new LocalFirehoseFactory(new File("lol"), "rofl", null), new PlumberSchool() - { - @Override - public Plumber findPlumber( - DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics - ) - { - return null; - } - }, null - ), null + new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, mapper), + new RealtimeIOConfig( + new LocalFirehoseFactory(new File("lol"), "rofl", null), + (schema, config, metrics) -> null, + null + ), + null ), null ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 0091c40b15d..029024a38bb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -599,7 +599,7 @@ public class IndexTaskTest File tmpDir = temporaryFolder.newFolder(); File tmpFile = File.createTempFile("druid", "index", tmpDir); - populateRollupTestData(tmpFile); + populateRollupTestData(tmpFile); IndexTask indexTask = new IndexTask( null, @@ -885,73 +885,95 @@ public class IndexTaskTest indexTask.run( new TaskToolbox( - null, new TaskActionClient() - { - @Override - public RetType submit(TaskAction taskAction) throws IOException - { - if (taskAction instanceof LockListAction) { - return (RetType) Collections.singletonList( - new TaskLock( - "", "", null, DateTimes.nowUtc().toString() - ) - ); - } + null, + new TaskActionClient() + { + @Override + public RetType submit(TaskAction taskAction) throws IOException + { + if (taskAction instanceof LockListAction) { + return (RetType) Collections.singletonList( + new TaskLock( + "", "", null, DateTimes.nowUtc().toString() + ) + ); + } - if (taskAction instanceof LockAcquireAction) { - return (RetType) new TaskLock( - "groupId", - "test", - ((LockAcquireAction) taskAction).getInterval(), - DateTimes.nowUtc().toString() - ); - } + if (taskAction instanceof LockAcquireAction) { + return (RetType) new TaskLock( + "groupId", + "test", + ((LockAcquireAction) taskAction).getInterval(), + DateTimes.nowUtc().toString() + ); + } - if (taskAction instanceof SegmentTransactionalInsertAction) { - return (RetType) new SegmentPublishResult( - ((SegmentTransactionalInsertAction) taskAction).getSegments(), - true - ); - } + if (taskAction instanceof SegmentTransactionalInsertAction) { + return (RetType) new SegmentPublishResult( + ((SegmentTransactionalInsertAction) taskAction).getSegments(), + true + ); + } - if (taskAction instanceof SegmentAllocateAction) { - SegmentAllocateAction action = (SegmentAllocateAction) taskAction; - Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp()); - ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0); - return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec); - } + if (taskAction instanceof SegmentAllocateAction) { + SegmentAllocateAction action = (SegmentAllocateAction) taskAction; + Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp()); + ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0); + return (RetType) new SegmentIdentifier(action.getDataSource(), interval, "latestVersion", shardSpec); + } - return null; - } - }, null, new DataSegmentPusher() - { - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } + return null; + } + }, + null, + new DataSegmentPusher() + { + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } - @Override - public String getPathForHadoop() - { - return null; - } + @Override + public String getPathForHadoop() + { + return null; + } - @Override - public DataSegment push(File file, DataSegment segment) throws IOException - { - segments.add(segment); - return segment; - } + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + segments.add(segment); + return segment; + } - @Override - public Map makeLoadSpec(URI uri) - { - throw new UnsupportedOperationException(); - } - }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), - indexIO, null, null, indexMergerV9, null, null, null, null + @Override + public Map makeLoadSpec(URI uri) + { + throw new UnsupportedOperationException(); + } + }, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + jsonMapper, + temporaryFolder.newFolder(), + indexIO, + null, + null, + indexMergerV9, + null, + null, + null, + null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index c40d8e30ec9..dadaef8657b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -958,22 +958,25 @@ public class RealtimeIndexTaskTest taskStorage, taskActionToolbox ); + IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( + null, + null, + null + ) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + }; final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( ImmutableMap., QueryRunnerFactory>of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate( - QueryRunner delegate, QueryToolChest> toolChest - ) - { - return delegate; - } - } - ), + new TimeseriesQueryQueryToolChest(queryRunnerDecorator), new TimeseriesQueryEngine(), new QueryWatcher() { @@ -1023,6 +1026,14 @@ public class RealtimeIndexTaskTest } }; final TestUtils testUtils = new TestUtils(); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, @@ -1038,17 +1049,7 @@ public class RealtimeIndexTaskTest MoreExecutors.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( - new SegmentLoaderLocalCacheManager( - null, - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Lists.newArrayList(); - } - }, testUtils.getTestObjectMapper() - ) + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) ), testUtils.getTestObjectMapper(), testUtils.getTestIndexIO(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 6e68e8f9952..b9d3abbc5e3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -140,101 +140,121 @@ public class SameIntervalMergeTaskTest mergeTask.run( new TaskToolbox( - null, new TaskActionClient() - { - @Override - public RetType submit(TaskAction taskAction) throws IOException - { - if (taskAction instanceof LockListAction) { - Assert.assertNotNull("taskLock should be acquired before list", taskLock); - return (RetType) Arrays.asList(taskLock); - } - if (taskAction instanceof SegmentListUsedAction) { - List segments = ImmutableList.of( - DataSegment.builder() - .dataSource(mergeTask.getDataSource()) - .interval(Intervals.of("2010-01-01/PT1H")) - .version("oldVersion") - .shardSpec(new LinearShardSpec(0)) - .build(), - DataSegment.builder() - .dataSource(mergeTask.getDataSource()) - .interval(Intervals.of("2010-01-01/PT1H")) - .version("oldVersion") - .shardSpec(new LinearShardSpec(0)) - .build(), - DataSegment.builder() - .dataSource(mergeTask.getDataSource()) - .interval(Intervals.of("2010-01-01/PT2H")) - .version("oldVersion") - .shardSpec(new LinearShardSpec(0)) - .build() - ); - return (RetType) segments; - } - if (taskAction instanceof SegmentInsertAction) { - publishCountDown.countDown(); - return null; - } + null, + new TaskActionClient() + { + @Override + public RetType submit(TaskAction taskAction) throws IOException + { + if (taskAction instanceof LockListAction) { + Assert.assertNotNull("taskLock should be acquired before list", taskLock); + return (RetType) Arrays.asList(taskLock); + } + if (taskAction instanceof SegmentListUsedAction) { + List segments = ImmutableList.of( + DataSegment.builder() + .dataSource(mergeTask.getDataSource()) + .interval(Intervals.of("2010-01-01/PT1H")) + .version("oldVersion") + .shardSpec(new LinearShardSpec(0)) + .build(), + DataSegment.builder() + .dataSource(mergeTask.getDataSource()) + .interval(Intervals.of("2010-01-01/PT1H")) + .version("oldVersion") + .shardSpec(new LinearShardSpec(0)) + .build(), + DataSegment.builder() + .dataSource(mergeTask.getDataSource()) + .interval(Intervals.of("2010-01-01/PT2H")) + .version("oldVersion") + .shardSpec(new LinearShardSpec(0)) + .build() + ); + return (RetType) segments; + } + if (taskAction instanceof SegmentInsertAction) { + publishCountDown.countDown(); + return null; + } - return null; - } - }, new NoopServiceEmitter(), new DataSegmentPusher() - { - @Deprecated - @Override - public String getPathForHadoop(String dataSource) - { - return getPathForHadoop(); - } + return null; + } + }, + new NoopServiceEmitter(), new DataSegmentPusher() + { + @Deprecated + @Override + public String getPathForHadoop(String dataSource) + { + return getPathForHadoop(); + } - @Override - public String getPathForHadoop() - { - return null; - } + @Override + public String getPathForHadoop() + { + return null; + } - @Override - public DataSegment push(File file, DataSegment segment) throws IOException - { - // the merged segment is pushed to storage - segments.add(segment); - return segment; - } - @Override - public Map makeLoadSpec(URI finalIndexZipFilePath) - { - return null; - } + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + // the merged segment is pushed to storage + segments.add(segment); + return segment; + } + @Override + public Map makeLoadSpec(URI finalIndexZipFilePath) + { + return null; + } - }, null, null, null, null, null, null, null, null, null, new SegmentLoader() - { - @Override - public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException - { - return false; - } + }, + null, + null, + null, + null, + null, + null, + null, + null, + null, + new SegmentLoader() + { + @Override + public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException + { + return false; + } - @Override - public Segment getSegment(DataSegment segment) throws SegmentLoadingException - { - return null; - } + @Override + public Segment getSegment(DataSegment segment) throws SegmentLoadingException + { + return null; + } - @Override - public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException - { - // dummy file to represent the downloaded segment's dir - return new File("" + segment.getShardSpec().getPartitionNum()); - } + @Override + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { + // dummy file to represent the downloaded segment's dir + return new File("" + segment.getShardSpec().getPartitionNum()); + } - @Override - public void cleanup(DataSegment segment) throws SegmentLoadingException - { - } - }, jsonMapper, temporaryFolder.newFolder(), - indexIO, null, null, EasyMock.createMock(IndexMergerV9.class), - null, null, null, null + @Override + public void cleanup(DataSegment segment) throws SegmentLoadingException + { + } + }, + jsonMapper, + temporaryFolder.newFolder(), + indexIO, + null, + null, + EasyMock.createMock(IndexMergerV9.class), + null, + null, + null, + null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 02dfdbbd050..80bf2947670 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -473,15 +473,15 @@ public class TaskSerdeTest ), new RealtimeIOConfig( new LocalFirehoseFactory(new File("lol"), "rofl", null), new PlumberSchool() - { - @Override - public Plumber findPlumber( - DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics - ) - { - return null; - } - }, + { + @Override + public Plumber findPlumber( + DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics + ) + { + return null; + } + }, null ), diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 59f21640036..40ab84887b3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -202,6 +202,14 @@ public class IngestSegmentFirehoseFactoryTest SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); EasyMock.replay(notifierFactory); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory( new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null, false, null, null), tac, @@ -277,17 +285,7 @@ public class IngestSegmentFirehoseFactoryTest null, // query executor service null, // monitor scheduler new SegmentLoaderFactory( - new SegmentLoaderLocalCacheManager( - null, - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Lists.newArrayList(); - } - }, MAPPER - ) + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, MAPPER) ), MAPPER, INDEX_IO, diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 25c8fe77e4f..19a854dfbb9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -296,6 +296,14 @@ public class IngestSegmentFirehoseFactoryTimelineTest }; SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); EasyMock.replay(notifierFactory); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory( new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null, false, null, null), new TaskActionClientFactory() @@ -318,17 +326,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest null, // query executor service null, // monitor scheduler new SegmentLoaderFactory( - new SegmentLoaderLocalCacheManager( - null, - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Lists.newArrayList(); - } - }, MAPPER - ) + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, MAPPER) ), MAPPER, INDEX_IO, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index cb6d21ff74d..73d0729202d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -89,18 +89,16 @@ public class RealtimeishTask extends AbstractTask Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2); // Push first segment - toolbox.getTaskActionClient() - .submit( - new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval1) - .version(lock1.getVersion()) - .build() - ) - ) - ); + SegmentInsertAction firstSegmentInsertAction = new SegmentInsertAction( + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval1) + .version(lock1.getVersion()) + .build() + ) + ); + toolbox.getTaskActionClient().submit(firstSegmentInsertAction); // Release first lock toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); @@ -110,18 +108,16 @@ public class RealtimeishTask extends AbstractTask Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3); // Push second segment - toolbox.getTaskActionClient() - .submit( - new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval2) - .version(lock2.getVersion()) - .build() - ) - ) - ); + SegmentInsertAction secondSegmentInsertAction = new SegmentInsertAction( + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval2) + .version(lock2.getVersion()) + .build() + ) + ); + toolbox.getTaskActionClient().submit(secondSegmentInsertAction); // Release second lock toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 8d38b8abc10..1a1b9616228 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -172,9 +172,8 @@ public class TaskLifecycleTest this.taskStorageType = taskStorageType; } - public final @Rule - TemporaryFolder temporaryFolder = new TemporaryFolder(); + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private static final Ordering byIntervalOrdering = new Ordering() { @@ -529,6 +528,14 @@ public class TaskLifecycleTest File tmpDir = temporaryFolder.newFolder(); taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; return new TaskToolboxFactory( taskConfig, tac, @@ -590,17 +597,7 @@ public class TaskLifecycleTest MoreExecutors.sameThreadExecutor(), // query executor service monitorScheduler, // monitor scheduler new SegmentLoaderFactory( - new SegmentLoaderLocalCacheManager( - null, - new SegmentLoaderConfig() - { - @Override - public List getLocations() - { - return Lists.newArrayList(); - } - }, new DefaultObjectMapper() - ) + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, new DefaultObjectMapper()) ), MAPPER, INDEX_IO, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java index b2f3a8be177..f459d35616f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java @@ -19,7 +19,6 @@ package io.druid.indexing.overlord.autoscaling; -import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -347,12 +346,8 @@ public class PendingTaskBasedProvisioningStrategyTest new TestZkWorker(testTask).toImmutable() ) ).times(2); - EasyMock.expect(runner.markWorkersLazy((Predicate) EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - Arrays.asList( - new TestZkWorker(testTask).getWorker() - ) - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.singletonList(new TestZkWorker(testTask).getWorker())); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); EasyMock.replay(runner); @@ -386,12 +381,8 @@ public class PendingTaskBasedProvisioningStrategyTest ) ).times(2); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()).times(2); - EasyMock.expect(runner.markWorkersLazy((Predicate) EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - Arrays.asList( - new TestZkWorker(testTask).toImmutable().getWorker() - ) - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.singletonList(new TestZkWorker(testTask).toImmutable().getWorker())); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); @@ -439,10 +430,8 @@ public class PendingTaskBasedProvisioningStrategyTest EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); - EasyMock.expect(runner.markWorkersLazy((Predicate) EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.emptyList() - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.emptyList()); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); @@ -486,10 +475,8 @@ public class PendingTaskBasedProvisioningStrategyTest EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); - EasyMock.expect(runner.markWorkersLazy((Predicate) EasyMock.anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.emptyList() - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.emptyList()); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java index 5701fa34ad7..e388173e418 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java @@ -280,11 +280,7 @@ public class SimpleProvisioningStrategyTest ) ).times(2); EasyMock.expect(runner.markWorkersLazy(EasyMock.>anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.singletonList( - new TestZkWorker(testTask).getWorker() - ) - ); + .andReturn(Collections.singletonList(new TestZkWorker(testTask).getWorker())); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); EasyMock.replay(runner); @@ -324,12 +320,8 @@ public class SimpleProvisioningStrategyTest ) ).times(2); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()).times(2); - EasyMock.expect(runner.markWorkersLazy(EasyMock.>anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.singletonList( - new TestZkWorker(testTask).getWorker() - ) - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.singletonList(new TestZkWorker(testTask).getWorker())); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); @@ -376,10 +368,8 @@ public class SimpleProvisioningStrategyTest ) ).times(2); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); - EasyMock.expect(runner.markWorkersLazy(EasyMock.>anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.emptyList() - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.emptyList()); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); @@ -422,10 +412,8 @@ public class SimpleProvisioningStrategyTest ) ).times(3); EasyMock.expect(runner.getLazyWorkers()).andReturn(Lists.newArrayList()); - EasyMock.expect(runner.markWorkersLazy(EasyMock.>anyObject(), EasyMock.anyInt())) - .andReturn( - Collections.emptyList() - ); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.emptyList()); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java index a98deebf75b..2993b224c85 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java @@ -41,6 +41,14 @@ public class EqualDistributionWithAffinityWorkerSelectStrategyTest new AffinityConfig(ImmutableMap.of("foo", ImmutableSet.of("localhost1", "localhost2", "localhost3")), false) ); + NoopTask noopTask = new NoopTask(null, 1, 0, null, null, null) + { + @Override + public String getDataSource() + { + return "foo"; + } + }; ImmutableWorkerInfo worker = strategy.findWorkerForTask( new RemoteTaskRunnerConfig(), ImmutableMap.of( @@ -73,14 +81,7 @@ public class EqualDistributionWithAffinityWorkerSelectStrategyTest DateTimes.nowUtc() ) ), - new NoopTask(null, 1, 0, null, null, null) - { - @Override - public String getDataSource() - { - return "foo"; - } - } + noopTask ); Assert.assertEquals("localhost1", worker.getWorker().getHost()); } diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index 1d160e99bff..1e147f1517f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -61,16 +61,16 @@ public class TaskAnnouncementTest new DataSchema("foo", null, new AggregatorFactory[0], null, new DefaultObjectMapper()), new RealtimeIOConfig( new LocalFirehoseFactory(new File("lol"), "rofl", null), new PlumberSchool() - { - @Override - public Plumber findPlumber( - DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics - ) - { - return null; - } + { + @Override + public Plumber findPlumber( + DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics + ) + { + return null; + } - }, + }, null ), null diff --git a/integration-tests/src/main/java/io/druid/testing/IntegrationTestingCuratorConfig.java b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingCuratorConfig.java index dac00f16be3..c2457e46e43 100644 --- a/integration-tests/src/main/java/io/druid/testing/IntegrationTestingCuratorConfig.java +++ b/integration-tests/src/main/java/io/druid/testing/IntegrationTestingCuratorConfig.java @@ -34,7 +34,7 @@ public class IntegrationTestingCuratorConfig extends CuratorConfig @Inject public IntegrationTestingCuratorConfig (IntegrationTestingConfig config) { - this.config = config; + this.config = config; } @Override diff --git a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java index fe0c0999909..e4231d3e221 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/CoordinatorResourceTestClient.java @@ -49,8 +49,9 @@ public class CoordinatorResourceTestClient @Inject CoordinatorResourceTestClient( - ObjectMapper jsonMapper, - @TestClient HttpClient httpClient, IntegrationTestingConfig config + ObjectMapper jsonMapper, + @TestClient HttpClient httpClient, + IntegrationTestingConfig config ) { this.jsonMapper = jsonMapper; diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java index 2d4e695f024..2039bca7708 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java @@ -58,9 +58,9 @@ public class OverlordResourceTestClient @Inject OverlordResourceTestClient( - ObjectMapper jsonMapper, - @TestClient HttpClient httpClient, - IntegrationTestingConfig config + ObjectMapper jsonMapper, + @TestClient HttpClient httpClient, + IntegrationTestingConfig config ) { this.jsonMapper = jsonMapper; diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java index 3d210c5e820..b5c3e762257 100644 --- a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java +++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java @@ -65,11 +65,10 @@ public class DruidTestModule implements Module @Provides @TestClient public HttpClient getHttpClient( - IntegrationTestingConfig config, - Lifecycle lifecycle, - @EscalatedClient HttpClient delegate - ) - throws Exception + IntegrationTestingConfig config, + Lifecycle lifecycle, + @EscalatedClient HttpClient delegate + ) throws Exception { if (config.getUsername() != null) { return new CredentialedHttpClient(new BasicCredentials(config.getUsername(), config.getPassword()), delegate); diff --git a/integration-tests/src/main/java/io/druid/testing/utils/LoggerListener.java b/integration-tests/src/main/java/io/druid/testing/utils/LoggerListener.java index 77ea6d063d9..9bb880f66c4 100644 --- a/integration-tests/src/main/java/io/druid/testing/utils/LoggerListener.java +++ b/integration-tests/src/main/java/io/druid/testing/utils/LoggerListener.java @@ -31,19 +31,19 @@ public class LoggerListener extends TestListenerAdapter @Override public void onTestFailure(ITestResult tr) { - LOG.info ("[%s] -- Test method failed", tr.getName()); + LOG.info ("[%s] -- Test method failed", tr.getName()); } @Override public void onTestSkipped(ITestResult tr) { - LOG.info ("[%s] -- Test method skipped", tr.getName()); + LOG.info ("[%s] -- Test method skipped", tr.getName()); } @Override public void onTestSuccess(ITestResult tr) { - LOG.info ("[%s] -- Test method passed", tr.getName()); + LOG.info ("[%s] -- Test method passed", tr.getName()); } @Override diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java index f5b30832e10..8b7f38757df 100644 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -42,16 +42,13 @@ import org.testng.xml.XmlTest; import java.net.URL; import java.util.List; -import java.util.concurrent.Callable; public class DruidTestRunnerFactory implements ITestRunnerFactory { private static final Logger LOG = new Logger(DruidTestRunnerFactory.class); @Override - public TestRunner newTestRunner( - ISuite suite, XmlTest test, List listeners - ) + public TestRunner newTestRunner(ISuite suite, XmlTest test, List listeners) { IConfiguration configuration = TestNG.getDefault().getConfiguration(); String outputDirectory = suite.getOutputDirectory(); @@ -72,13 +69,13 @@ public class DruidTestRunnerFactory implements ITestRunnerFactory { protected DruidTestRunner( - IConfiguration configuration, - ISuite suite, - XmlTest test, - String outputDirectory, - IAnnotationFinder finder, - boolean skipFailedInvocationCounts, - List invokedMethodListeners + IConfiguration configuration, + ISuite suite, + XmlTest test, + String outputDirectory, + IAnnotationFinder finder, + boolean skipFailedInvocationCounts, + List invokedMethodListeners ) { super(configuration, suite, test, outputDirectory, finder, skipFailedInvocationCounts, invokedMethodListeners); @@ -122,38 +119,22 @@ public class DruidTestRunnerFactory implements ITestRunnerFactory { final StatusResponseHandler handler = new StatusResponseHandler(Charsets.UTF_8); RetryUtil.retryUntilTrue( - new Callable() - { - @Override - public Boolean call() throws Exception - { + () -> { try { StatusResponseHolder response = client.go( - new Request( - HttpMethod.GET, - new URL( - StringUtils.format( - "%s/status", - host - ) - ) - ), - handler + new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status", host))), + handler ).get(); LOG.info("%s %s", response.getStatus(), response.getContent()); - if (response.getStatus().equals(HttpResponseStatus.OK)) { - return true; - } else { - return false; - } + return response.getStatus().equals(HttpResponseStatus.OK); } catch (Throwable e) { LOG.error(e, ""); return false; } - } - }, "Waiting for instance to be ready: [" + host + "]" + }, + "Waiting for instance to be ready: [" + host + "]" ); } } diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java index 82b83fb983c..148482cd88b 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractIndexerTest.java @@ -55,17 +55,17 @@ public abstract class AbstractIndexerTest protected void unloadAndKillData(final String dataSource) throws Exception { - ArrayList intervals = coordinator.getSegmentIntervals(dataSource); + ArrayList intervals = coordinator.getSegmentIntervals(dataSource); - // each element in intervals has this form: - // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z - // we'll sort the list (ISO dates have lexicographic order) - // then delete segments from the 1st date in the first string - // to the 2nd date in the last string - Collections.sort (intervals); - String first = intervals.get(0).split("/")[0]; - String last = intervals.get(intervals.size() - 1).split("/")[1]; - unloadAndKillData (dataSource, first, last); + // each element in intervals has this form: + // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z + // we'll sort the list (ISO dates have lexicographic order) + // then delete segments from the 1st date in the first string + // to the 2nd date in the last string + Collections.sort (intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + unloadAndKillData (dataSource, first, last); } protected void unloadAndKillData(final String dataSource, String start, String end) throws Exception @@ -92,15 +92,13 @@ public abstract class AbstractIndexerTest protected void waitForAllTasksToComplete() { RetryUtil.retryUntilTrue( - new Callable() - { - @Override - public Boolean call() throws Exception - { - return (indexer.getPendingTasks().size() + indexer.getRunningTasks().size() + indexer.getWaitingTasks() - .size()) == 0; - } - }, "Waiting for Tasks Completion" + () -> { + int numTasks = indexer.getPendingTasks().size() + + indexer.getRunningTasks().size() + + indexer.getWaitingTasks().size(); + return numTasks == 0; + }, + "Waiting for Tasks Completion" ); } diff --git a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java index bda0f98ba05..bee564ba3f6 100644 --- a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java @@ -19,7 +19,6 @@ package io.druid.java.util.common; -import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -97,16 +96,7 @@ public class JodaUtils public static boolean overlaps(final Interval i, Iterable intervals) { - return Iterables.any( - intervals, new Predicate() - { - @Override - public boolean apply(Interval input) - { - return input.overlaps(i); - } - } - ); + return Iterables.any(intervals, input -> input.overlaps(i)); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index efbab021390..dccefad4c30 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -40,16 +40,7 @@ public class ConcatSequence implements Sequence @Override public OutType accumulate(OutType initValue, final Accumulator accumulator) { - return baseSequences.accumulate( - initValue, new Accumulator>() - { - @Override - public OutType accumulate(OutType accumulated, Sequence in) - { - return in.accumulate(accumulated, accumulator); - } - } - ); + return baseSequences.accumulate(initValue, (accumulated, in) -> in.accumulate(accumulated, accumulator)); } @Override diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java index 5e7f005a4b8..1c9a2b4a7b9 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java @@ -92,10 +92,9 @@ final class LimitedSequence extends YieldingSequenceBase @Override public boolean isDone() { - return subYielder.isDone() || ( - !limitedAccumulator.withinThreshold() && (!limitedAccumulator.yielded() - || limitedAccumulator.isInterruptYield()) - ); + return subYielder.isDone() || + (!limitedAccumulator.withinThreshold() && + (!limitedAccumulator.yielded() || limitedAccumulator.isInterruptYield())); } @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java b/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java index 1a1be9f6566..292addfc65a 100644 --- a/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/GranularityTest.java @@ -38,429 +38,753 @@ import java.util.NoSuchElementException; public class GranularityTest { - final Granularity SECOND = Granularities.SECOND; - final Granularity MINUTE = Granularities.MINUTE; - final Granularity HOUR = Granularities.HOUR; - final Granularity SIX_HOUR = Granularities.SIX_HOUR; - final Granularity FIFTEEN_MINUTE = Granularities.FIFTEEN_MINUTE; - final Granularity DAY = Granularities.DAY; - final Granularity WEEK = Granularities.WEEK; - final Granularity MONTH = Granularities.MONTH; - final Granularity YEAR = Granularities.YEAR; + final Granularity SECOND = Granularities.SECOND; + final Granularity MINUTE = Granularities.MINUTE; + final Granularity HOUR = Granularities.HOUR; + final Granularity SIX_HOUR = Granularities.SIX_HOUR; + final Granularity FIFTEEN_MINUTE = Granularities.FIFTEEN_MINUTE; + final Granularity DAY = Granularities.DAY; + final Granularity WEEK = Granularities.WEEK; + final Granularity MONTH = Granularities.MONTH; + final Granularity YEAR = Granularities.YEAR; - @Test - public void testHiveFormat() - { - PathDate[] secondChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "dt=2011-03-15-20-50-43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "/dt=2011-03-15-20-50-43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "valid/dt=2011-03-15-20-50-43/Test1"), - new PathDate(null, null, "valid/dt=2011-03-15-20-50/Test2"), - new PathDate(null, null, "valid/dt=2011-03-15-20/Test3"), - new PathDate(null, null, "valid/dt=2011-03-15/Test4"), - new PathDate(null, null, "valid/dt=2011-03/Test5"), - new PathDate(null, null, "valid/dt=2011/Test6"), - new PathDate(null, null, "null/dt=----/Test7"), - new PathDate(null, null, "null/10-2011-23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-20-20-42-72/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-20-42-90-24/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-33-20-42-24/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-13-20-20-42-24/Test11"), + @Test + public void testHiveFormat() + { + PathDate[] secondChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "dt=2011-03-15-20-50-43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "/dt=2011-03-15-20-50-43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "valid/dt=2011-03-15-20-50-43/Test1" + ), + new PathDate(null, null, "valid/dt=2011-03-15-20-50/Test2"), + new PathDate(null, null, "valid/dt=2011-03-15-20/Test3"), + new PathDate(null, null, "valid/dt=2011-03-15/Test4"), + new PathDate(null, null, "valid/dt=2011-03/Test5"), + new PathDate(null, null, "valid/dt=2011/Test6"), + new PathDate(null, null, "null/dt=----/Test7"), + new PathDate(null, null, "null/10-2011-23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-20-20-42-72/Test11"), + new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-20-42-90-24/Test11"), + new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-10-33-20-42-24/Test11"), + new PathDate(null, IllegalFieldValueException.class, "error/dt=2011-13-20-20-42-24/Test11"), }; - checkToDate(SECOND, Granularity.Formatter.HIVE, secondChecks); - } + checkToDate(SECOND, Granularity.Formatter.HIVE, secondChecks); + } - @Test - public void testSecondToDate() - { - PathDate[] secondChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + @Test + public void testSecondToDate() + { + PathDate[] secondChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 43, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), + new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), + new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(SECOND, Granularity.Formatter.DEFAULT, secondChecks); - } + checkToDate(SECOND, Granularity.Formatter.DEFAULT, secondChecks); + } - @Test - public void testMinuteToDate() - { + @Test + public void testMinuteToDate() + { - PathDate[] minuteChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 20, 20, 42, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + PathDate[] minuteChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), + new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 20, 20, 42, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(MINUTE, Granularity.Formatter.DEFAULT, minuteChecks); - } + checkToDate(MINUTE, Granularity.Formatter.DEFAULT, minuteChecks); + } - @Test - public void testFifteenMinuteToDate() - { + @Test + public void testFifteenMinuteToDate() + { - PathDate[] minuteChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 3, 15, 20, 00, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=00/Test2a"), - new PathDate(new DateTime(2011, 3, 15, 20, 00, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=14/Test2b"), - new PathDate(new DateTime(2011, 3, 15, 20, 15, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=15/Test2c"), - new PathDate(new DateTime(2011, 3, 15, 20, 15, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=29/Test2d"), - new PathDate(new DateTime(2011, 3, 15, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=30/Test2e"), - new PathDate(new DateTime(2011, 3, 15, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=44/Test2f"), - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=45/Test2g"), - new PathDate(new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=59/Test2h"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 20, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + PathDate[] minuteChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 00, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=00/Test2a" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 00, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=14/Test2b" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 15, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=15/Test2c" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 15, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=29/Test2d" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=30/Test2e" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=44/Test2f" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=45/Test2g" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 45, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=59/Test2h" + ), + new PathDate(null, null, "valid/y=2011/m=03/d=15/H=20/Test3"), + new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 20, 20, 30, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(FIFTEEN_MINUTE, Granularity.Formatter.DEFAULT, minuteChecks); - } + checkToDate(FIFTEEN_MINUTE, Granularity.Formatter.DEFAULT, minuteChecks); + } - @Test - public void testHourToDate() - { - PathDate[] hourChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 20, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(new DateTime(2011, 10, 20, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + @Test + public void testHourToDate() + { + PathDate[] hourChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/Test3" + ), + new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 20, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate( + new DateTime(2011, 10, 20, 20, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(HOUR, Granularity.Formatter.DEFAULT, hourChecks); - } + checkToDate(HOUR, Granularity.Formatter.DEFAULT, hourChecks); + } - @Test - public void testSixHourToDate() - { - PathDate[] hourChecks = { - new PathDate(new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 20, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(new DateTime(2011, 10, 20, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=00/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=02/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 6, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=06/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 6, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=11/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 12, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=12/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 12, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=13/M=90/S=24/Test12"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + @Test + public void testSixHourToDate() + { + PathDate[] hourChecks = { + new PathDate( + new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 3, 15, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/Test3" + ), + new PathDate(null, null, "valid/y=2011/m=03/d=15/Test4"), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 20, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate( + new DateTime(2011, 10, 20, 18, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=00/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=02/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 6, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=06/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 6, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=11/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 12, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=12/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 12, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=13/M=90/S=24/Test12" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(SIX_HOUR, Granularity.Formatter.DEFAULT, hourChecks); - } + checkToDate(SIX_HOUR, Granularity.Formatter.DEFAULT, hourChecks); + } - @Test - public void testDayToDate() - { - PathDate[] dayChecks = { - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(null, null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + @Test + public void testDayToDate() + { + PathDate[] dayChecks = { + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/Test3" + ), + new PathDate( + new DateTime(2011, 3, 15, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/Test4" + ), + new PathDate(null, null, "valid/y=2011/m=03/Test5"), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate( + new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 20, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(DAY, Granularity.Formatter.DEFAULT, dayChecks); - } + checkToDate(DAY, Granularity.Formatter.DEFAULT, dayChecks); + } - @Test - public void testMonthToDate() - { - PathDate[] monthChecks = { - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/Test5"), - new PathDate(null, null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; + @Test + public void testMonthToDate() + { + PathDate[] monthChecks = { + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/Test3" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/Test4" + ), + new PathDate( + new DateTime(2011, 3, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/Test5" + ), + new PathDate(null, null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate(null, null, "null/m=10/y=2011/d=23/Test8"), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate( + new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13" + ), + new PathDate( + new DateTime(2011, 10, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14" + ), + new PathDate(null, IllegalFieldValueException.class, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") + }; - checkToDate(MONTH, Granularity.Formatter.DEFAULT, monthChecks); - } + checkToDate(MONTH, Granularity.Formatter.DEFAULT, monthChecks); + } - @Test - public void testYearToDate() - { - PathDate[] yearChecks = { - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/Test2"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/Test3"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/Test4"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/Test5"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/Test6"), - new PathDate(null, null, "null/y=/m=/d=/Test7"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "null/m=10/y=2011/d=23/Test8"), - new PathDate(null, null, "null/Test9"), - new PathDate(null, null, ""), //Test10 Intentionally empty. - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14"), - new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15") - }; - checkToDate(YEAR, Granularity.Formatter.DEFAULT, yearChecks); - } + @Test + public void testYearToDate() + { + PathDate[] yearChecks = { + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/Test2" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/Test3" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/Test4" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/Test5" + ), + new PathDate(new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/Test6"), + new PathDate(null, null, "null/y=/m=/d=/Test7"), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "null/m=10/y=2011/d=23/Test8" + ), + new PathDate(null, null, "null/Test9"), + new PathDate(null, null, ""), //Test10 Intentionally empty. + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=42/S=72/Test11" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=20/M=90/S=24/Test12" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=20/H=42/M=42/S=24/Test13" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=10/d=33/H=20/M=42/S=24/Test14" + ), + new PathDate( + new DateTime(2011, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), + null, + "error/y=2011/m=13/d=20/H=20/M=42/S=24/Test15" + ) + }; + checkToDate(YEAR, Granularity.Formatter.DEFAULT, yearChecks); + } - private void checkToDate(Granularity granularity, Granularity.Formatter formatter, PathDate[] checks) - { - for (PathDate pd : checks) { - if (pd.exception == null) { - // check if path returns expected date - Assert.assertEquals( - StringUtils.format( - "[%s,%s] Expected path %s to return date %s", - granularity, - formatter, - pd.path, - pd.date - ), - pd.date, - granularity.toDate(pd.path, formatter) - ); + private void checkToDate(Granularity granularity, Granularity.Formatter formatter, PathDate[] checks) + { + for (PathDate pd : checks) { + if (pd.exception == null) { + // check if path returns expected date + Assert.assertEquals( + StringUtils.format( + "[%s,%s] Expected path %s to return date %s", + granularity, + formatter, + pd.path, + pd.date + ), + pd.date, + granularity.toDate(pd.path, formatter) + ); - if (formatter.equals(Granularity.Formatter.DEFAULT)) { - Assert.assertEquals( - StringUtils.format( - "[%s] Expected toDate(%s) to return the same as toDate(%s, DEFAULT)", - granularity, - pd.path, - pd.path - ), - granularity.toDate(pd.path), granularity.toDate(pd.path, formatter) - ); - } - - if (pd.date != null) { - // check if formatter is readable by toDate - Assert.assertEquals( - StringUtils.format( - "[%s,%s] Expected date %s to return date %s", - granularity, - formatter, - pd.date, - pd.date - ), - pd.date, - granularity.toDate(granularity.getFormatter(formatter).print(pd.date) + "/", formatter) - ); - } - } else { - boolean flag = false; - try { - granularity.toDate(pd.path, formatter); - } - catch (Exception e) { - if (e.getClass() == pd.exception) { - flag = true; - } - } - - Assert.assertTrue( - StringUtils.format( - "[%s,%s] Expected exception %s for path: %s", - granularity, - formatter, - pd.exception, - pd.path - ), - flag - ); - } + if (formatter.equals(Granularity.Formatter.DEFAULT)) { + Assert.assertEquals( + StringUtils.format( + "[%s] Expected toDate(%s) to return the same as toDate(%s, DEFAULT)", + granularity, + pd.path, + pd.path + ), + granularity.toDate(pd.path), granularity.toDate(pd.path, formatter) + ); } - } - - @Test - public void testBucket() - { - DateTime dt = DateTimes.of("2011-02-03T04:05:06.100"); - - Assert.assertEquals(Intervals.of("2011-01-01/2012-01-01"), YEAR.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-02-01/2011-03-01"), MONTH.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-01-31/2011-02-07"), WEEK.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-02-03/2011-02-04"), DAY.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-02-03T04/2011-02-03T05"), HOUR.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-02-03T04:05:00/2011-02-03T04:06:00"), MINUTE.bucket(dt)); - Assert.assertEquals(Intervals.of("2011-02-03T04:05:06/2011-02-03T04:05:07"), SECOND.bucket(dt)); - - // Test with aligned DateTime - Assert.assertEquals(Intervals.of("2011-01-01/2011-01-02"), DAY.bucket(DateTimes.of("2011-01-01"))); - } - - @Test - public void testTruncate() throws Exception - { - DateTime date = DateTimes.of("2011-03-15T22:42:23.898"); - Assert.assertEquals(DateTimes.of("2011-01-01T00:00:00.000"), YEAR.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-01T00:00:00.000"), MONTH.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-14T00:00:00.000"), WEEK.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-15T00:00:00.000"), DAY.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-15T22:00:00.000"), HOUR.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-15T22:42:00.000"), MINUTE.bucketStart(date)); - Assert.assertEquals(DateTimes.of("2011-03-15T22:42:23.000"), SECOND.bucketStart(date)); - } - - @Test - public void testGetIterable() throws Exception - { - DateTime start = DateTimes.of("2011-01-01T00:00:00"); - DateTime end = DateTimes.of("2011-01-14T00:00:00"); - - Iterator intervals = DAY.getIterable(new Interval(start, end)).iterator(); - - Assert.assertEquals(Intervals.of("2011-01-01/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-02/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-03/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-04/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-05/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-06/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-07/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-08/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-09/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-10/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-11/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-12/P1d"), intervals.next()); - Assert.assertEquals(Intervals.of("2011-01-13/P1d"), intervals.next()); + if (pd.date != null) { + // check if formatter is readable by toDate + Assert.assertEquals( + StringUtils.format( + "[%s,%s] Expected date %s to return date %s", + granularity, + formatter, + pd.date, + pd.date + ), + pd.date, + granularity.toDate(granularity.getFormatter(formatter).print(pd.date) + "/", formatter) + ); + } + } else { + boolean flag = false; try { - intervals.next(); + granularity.toDate(pd.path, formatter); } - catch (NoSuchElementException e) { - Assert.assertTrue(true); + catch (Exception e) { + if (e.getClass() == pd.exception) { + flag = true; + } } - } - @Test - public void testCustomPeriodToDate() + Assert.assertTrue( + StringUtils.format( + "[%s,%s] Expected exception %s for path: %s", + granularity, + formatter, + pd.exception, + pd.path + ), + flag + ); + } + } + } + + @Test + public void testBucket() + { + DateTime dt = DateTimes.of("2011-02-03T04:05:06.100"); + + Assert.assertEquals(Intervals.of("2011-01-01/2012-01-01"), YEAR.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-02-01/2011-03-01"), MONTH.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-01-31/2011-02-07"), WEEK.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-02-03/2011-02-04"), DAY.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-02-03T04/2011-02-03T05"), HOUR.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-02-03T04:05:00/2011-02-03T04:06:00"), MINUTE.bucket(dt)); + Assert.assertEquals(Intervals.of("2011-02-03T04:05:06/2011-02-03T04:05:07"), SECOND.bucket(dt)); + + // Test with aligned DateTime + Assert.assertEquals(Intervals.of("2011-01-01/2011-01-02"), DAY.bucket(DateTimes.of("2011-01-01"))); + } + + @Test + public void testTruncate() throws Exception + { + DateTime date = DateTimes.of("2011-03-15T22:42:23.898"); + Assert.assertEquals(DateTimes.of("2011-01-01T00:00:00.000"), YEAR.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-01T00:00:00.000"), MONTH.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-14T00:00:00.000"), WEEK.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-15T00:00:00.000"), DAY.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-15T22:00:00.000"), HOUR.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-15T22:42:00.000"), MINUTE.bucketStart(date)); + Assert.assertEquals(DateTimes.of("2011-03-15T22:42:23.000"), SECOND.bucketStart(date)); + } + + @Test + public void testGetIterable() throws Exception + { + DateTime start = DateTimes.of("2011-01-01T00:00:00"); + DateTime end = DateTimes.of("2011-01-14T00:00:00"); + + Iterator intervals = DAY.getIterable(new Interval(start, end)).iterator(); + + Assert.assertEquals(Intervals.of("2011-01-01/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-02/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-03/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-04/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-05/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-06/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-07/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-08/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-09/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-10/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-11/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-12/P1d"), intervals.next()); + Assert.assertEquals(Intervals.of("2011-01-13/P1d"), intervals.next()); + + try { + intervals.next(); + } + catch (NoSuchElementException e) { + Assert.assertTrue(true); + } + } + + @Test + public void testCustomPeriodToDate() + { + PathDate[] customChecks = { + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), + null, + "y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), + null, + "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0" + ), + new PathDate( + new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), + null, + "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1" + ) + }; + checkToDate( + new PeriodGranularity(new Period("PT2S"), null, DateTimeZone.UTC), + Granularity.Formatter.DEFAULT, + customChecks + ); + } + + @Test + public void testCustomNestedPeriodFail() + { + try { + Period p = Period.years(6).withMonths(3).withSeconds(23); + GranularityType.fromPeriod(p); + Assert.fail("Complicated period creation should fail b/c of unsupported granularity type."); + } + catch (IAE e) { + // pass + } + } + + private static class PathDate + { + public final String path; + public final DateTime date; + + public final Class exception; + + private PathDate(DateTime date, Class exception, String path) { - PathDate[] customChecks = { - new PathDate(new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), null, "y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), null, "/y=2011/m=03/d=15/H=20/M=50/S=43/Test0"), - new PathDate(new DateTime(2011, 3, 15, 20, 50, 42, 0, ISOChronology.getInstanceUTC()), null, "valid/y=2011/m=03/d=15/H=20/M=50/S=43/Test1") - }; - checkToDate(new PeriodGranularity(new Period("PT2S"), null, DateTimeZone.UTC), Granularity.Formatter.DEFAULT, customChecks); + this.path = path; + this.date = date; + this.exception = exception; } - @Test - public void testCustomNestedPeriodFail() - { - try { - Period p = Period.years(6).withMonths(3).withSeconds(23); - GranularityType.fromPeriod(p); - Assert.fail("Complicated period creation should fail b/c of unsupported granularity type."); - } - catch (IAE e) { - // pass - } - } - - private static class PathDate - { - public final String path; - public final DateTime date; - - public final Class exception; - - private PathDate(DateTime date, Class exception, String path) - { - this.path = path; - this.date = date; - this.exception = exception; - } - - } + } } diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java b/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java index 15ac7085aad..2e24d2ed69a 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java @@ -56,23 +56,24 @@ public class SequenceTestHelper { Iterator numsIter = nums.iterator(); Yielder yielder = seq.toYielder( - 0, new YieldingAccumulator() - { - final Iterator valsIter = nums.iterator(); - int count = 0; + 0, + new YieldingAccumulator() + { + final Iterator valsIter = nums.iterator(); + int count = 0; - @Override - public Integer accumulate(Integer accumulated, Integer in) - { - if (++count >= numToTake) { - count = 0; - yield(); + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + if (++count >= numToTake) { + count = 0; + yield(); + } + + Assert.assertEquals(prefix, valsIter.next(), in); + return accumulated + in; + } } - - Assert.assertEquals(prefix, valsIter.next(), in); - return accumulated + in; - } - } ); int expectedSum = 0; @@ -105,17 +106,18 @@ public class SequenceTestHelper } int sum = seq.accumulate( - 0, new Accumulator() - { - final Iterator valsIter = nums.iterator(); + 0, + new Accumulator() + { + final Iterator valsIter = nums.iterator(); - @Override - public Integer accumulate(Integer accumulated, Integer in) - { - Assert.assertEquals(prefix, valsIter.next(), in); - return accumulated + in; - } - } + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + Assert.assertEquals(prefix, valsIter.next(), in); + return accumulated + in; + } + } ); Assert.assertEquals(prefix, expectedSum, sum); diff --git a/processing/src/main/java/io/druid/jackson/JodaStuff.java b/processing/src/main/java/io/druid/jackson/JodaStuff.java index 43e569ab9e7..88af3c4dbcd 100644 --- a/processing/src/main/java/io/druid/jackson/JodaStuff.java +++ b/processing/src/main/java/io/druid/jackson/JodaStuff.java @@ -89,30 +89,30 @@ class JodaStuff private static class DateTimeDeserializer extends StdDeserializer { - public DateTimeDeserializer() - { - super(DateTime.class); - } + public DateTimeDeserializer() + { + super(DateTime.class); + } - @Override - public DateTime deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException - { - JsonToken t = jp.getCurrentToken(); - if (t == JsonToken.VALUE_NUMBER_INT) { - return DateTimes.utc(jp.getLongValue()); - } - if (t == JsonToken.VALUE_STRING) { - String str = jp.getText().trim(); - if (str.length() == 0) { // [JACKSON-360] - return null; - } - // make sure to preserve time zone information when parsing timestamps - return ISODateTimeFormat.dateTimeParser() - .withOffsetParsed() - .parseDateTime(str); - } - throw ctxt.mappingException(getValueClass()); + @Override + public DateTime deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException, JsonProcessingException + { + JsonToken t = jp.getCurrentToken(); + if (t == JsonToken.VALUE_NUMBER_INT) { + return DateTimes.utc(jp.getLongValue()); } + if (t == JsonToken.VALUE_STRING) { + String str = jp.getText().trim(); + if (str.length() == 0) { // [JACKSON-360] + return null; + } + // make sure to preserve time zone information when parsing timestamps + return ISODateTimeFormat.dateTimeParser() + .withOffsetParsed() + .parseDateTime(str); + } + throw ctxt.mappingException(getValueClass()); + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java index 8a73efca0db..c7a9a276e35 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/DoubleLastAggregatorFactory.java @@ -56,7 +56,7 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory public DoubleLastAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName - ) + ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); diff --git a/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java index 9bb88ec91a5..969b614af8d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/FloatLastAggregatorFactory.java @@ -56,7 +56,7 @@ public class FloatLastAggregatorFactory extends AggregatorFactory public FloatLastAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName - ) + ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); diff --git a/processing/src/main/java/io/druid/query/extraction/IdentityExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/IdentityExtractionFn.java index d0158e7f27e..1bf700feaaf 100644 --- a/processing/src/main/java/io/druid/query/extraction/IdentityExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/IdentityExtractionFn.java @@ -73,13 +73,13 @@ public class IdentityExtractionFn implements ExtractionFn @Override public String toString() { - return "Identity"; + return "Identity"; } @Override public boolean equals(Object o) { - return o != null && o instanceof IdentityExtractionFn; + return o != null && o instanceof IdentityExtractionFn; } public static final IdentityExtractionFn getInstance() diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java index 4723f831dba..07f2a72c823 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ByteBufferHashTable.java @@ -292,7 +292,7 @@ public class ByteBufferHashTable final int startBucket = keyHash % buckets; int bucket = startBucket; -outer: + outer: while (true) { final int bucketOffset = bucket * bucketSizeWithHash; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index ab6486559f5..48c07f1a88c 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -525,7 +525,7 @@ public class RowBasedGrouperHelper } private static class InputRawSupplierColumnSelectorStrategyFactory - implements ColumnSelectorStrategyFactory + implements ColumnSelectorStrategyFactory { @Override public InputRawSupplierColumnSelectorStrategy makeColumnSelectorStrategy( diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java index 0a03f8a652e..ae0250c6f92 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/column/DoubleGroupByColumnSelectorStrategy.java @@ -45,7 +45,7 @@ public class DoubleGroupByColumnSelectorStrategy implements GroupByColumnSelecto @Override public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] values) { - values[columnIndex] = selector.getDouble(); + values[columnIndex] = selector.getDouble(); } @Override diff --git a/processing/src/main/java/io/druid/query/lookup/LookupSnapshotTaker.java b/processing/src/main/java/io/druid/query/lookup/LookupSnapshotTaker.java index 6f65e14ddd1..3d67124610f 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupSnapshotTaker.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupSnapshotTaker.java @@ -83,7 +83,7 @@ public class LookupSnapshotTaker public synchronized void takeSnapshot(List lookups) { try { - objectMapper.writeValue(persistFile, lookups); + objectMapper.writeValue(persistFile, lookups); } catch (IOException e) { throw new ISE(e, "Exception during serialization of lookups using file [%s]", persistFile.getAbsolutePath()); diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index b793bc221ef..fbc97bd8ad3 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -98,8 +98,7 @@ public class PooledTopNAlgorithm new Generic2AggPooledTopNScannerPrototype(); private static final Historical1AggPooledTopNScanner defaultHistorical1SimpleDoubleAggScanner = new Historical1SimpleDoubleAggPooledTopNScannerPrototype(); - private static final - Historical1AggPooledTopNScanner defaultHistoricalSingleValueDimSelector1SimpleDoubleAggScanner = + private static final Historical1AggPooledTopNScanner defaultHistoricalSingleValueDimSelector1SimpleDoubleAggScanner = new HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype(); private interface ScanAndAggregate diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 1821a502687..86c3a414ed9 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -95,17 +95,17 @@ public class TopNQueryBuilder public TopNQueryBuilder(final TopNQuery query) { - this.dataSource = query.getDataSource(); - this.virtualColumns = query.getVirtualColumns(); - this.dimensionSpec = query.getDimensionSpec(); - this.topNMetricSpec = query.getTopNMetricSpec(); - this.threshold = query.getThreshold(); - this.querySegmentSpec = query.getQuerySegmentSpec(); - this.dimFilter = query.getDimensionsFilter(); - this.granularity = query.getGranularity(); - this.aggregatorSpecs = query.getAggregatorSpecs(); - this.postAggregatorSpecs = query.getPostAggregatorSpecs(); - this.context = query.getContext(); + this.dataSource = query.getDataSource(); + this.virtualColumns = query.getVirtualColumns(); + this.dimensionSpec = query.getDimensionSpec(); + this.topNMetricSpec = query.getTopNMetricSpec(); + this.threshold = query.getThreshold(); + this.querySegmentSpec = query.getQuerySegmentSpec(); + this.dimFilter = query.getDimensionsFilter(); + this.granularity = query.getGranularity(); + this.aggregatorSpecs = query.getAggregatorSpecs(); + this.postAggregatorSpecs = query.getPostAggregatorSpecs(); + this.context = query.getContext(); } public DataSource getDataSource() diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java index ff526e6662a..18604eadcf3 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java @@ -138,7 +138,9 @@ public final class DimensionHandlerUtils * @return An array of ColumnSelectorPlus objects, in the order of the columns specified in dimensionSpecs */ public static + //CHECKSTYLE.OFF: Indentation ColumnSelectorPlus[] createColumnSelectorPluses( + //CHECKSTYLE.ON: Indentation ColumnSelectorStrategyFactory strategyFactory, List dimensionSpecs, ColumnSelectorFactory columnSelectorFactory diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index d7d81d3ef54..7408c58f506 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -555,7 +555,7 @@ public class StringDimensionIndexer implements DimensionIndexer // these buffers are used by dictIdSeeker in mergeBitmaps() below. The iterator is created and only used // in writeMergedValueMetadata(), but the buffers are still used until after mergeBitmaps(). Closeable toCloseDictionaryMergeIterator = dictionaryMergeIterator; - Closeable dimValsMappedUnmapper = new Closeable() - { - @Override - public void close() - { - ByteBufferUtils.unmap(dimValsMapped); - } - }) { + Closeable dimValsMappedUnmapper = () -> ByteBufferUtils.unmap(dimValsMapped) + ) { Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java index 8ac5b84be10..972da01b829 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java @@ -65,22 +65,22 @@ public class BlockLayoutIndexedFloatSupplier implements Supplier final int rem = sizePer - 1; final boolean powerOf2 = sizePer == (1 << div); if (powerOf2) { - return new BlockLayoutIndexedFloats() + return new BlockLayoutIndexedFloats() + { + @Override + public float get(int index) { - @Override - public float get(int index) - { - // optimize division and remainder for powers of 2 - final int bufferNum = index >> div; + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - final int bufferIndex = index & rem; - return floatBuffer.get(floatBuffer.position() + bufferIndex); + if (bufferNum != currIndex) { + loadBuffer(bufferNum); } - }; + + final int bufferIndex = index & rem; + return floatBuffer.get(floatBuffer.position() + bufferIndex); + } + }; } else { return new BlockLayoutIndexedFloats(); } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedIterable.java b/processing/src/main/java/io/druid/segment/data/IndexedIterable.java index 2f6ad2bd3b3..9068c845b2f 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedIterable.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedIterable.java @@ -33,9 +33,7 @@ public class IndexedIterable implements Iterable return new IndexedIterable(indexed); } - public IndexedIterable( - Indexed indexed - ) + public IndexedIterable(Indexed indexed) { this.indexed = indexed; } diff --git a/processing/src/main/java/io/druid/segment/data/ReadableOffset.java b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java index 753df459da0..8c56d2ee63e 100644 --- a/processing/src/main/java/io/druid/segment/data/ReadableOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java @@ -31,7 +31,7 @@ import io.druid.query.monomorphicprocessing.HotLoopCallee; */ public interface ReadableOffset extends HotLoopCallee { - @CalledFromHotLoop - int getOffset(); + @CalledFromHotLoop + int getOffset(); } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index 01447cb83bd..2b38f5ae70d 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -104,20 +104,21 @@ public class ComplexColumnPartSerde implements ColumnPartSerde public ComplexColumnPartSerde build() { return new ComplexColumnPartSerde( - typeName, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } + typeName, + new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel, smoosher); - } - } + @Override + public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + delegate.writeToChannel(channel, smoosher); + } + } ); } } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index cb86f47957c..5d54c573616 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -302,15 +302,13 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde rMultiValuedColumn = null; } - builder.setHasMultipleValues(hasMultipleValues) - .setDictionaryEncodedColumn( - new DictionaryEncodedColumnSupplier( - rDictionary, - rSingleValuedColumn, - rMultiValuedColumn, - columnConfig.columnCacheSizeBytes() - ) - ); + DictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier = new DictionaryEncodedColumnSupplier( + rDictionary, + rSingleValuedColumn, + rMultiValuedColumn, + columnConfig.columnCacheSizeBytes() + ); + builder.setHasMultipleValues(hasMultipleValues).setDictionaryEncodedColumn(dictionaryEncodedColumnSupplier); GenericIndexed rBitmaps = GenericIndexed.read( buffer, bitmapSerdeFactory.getObjectStrategy(), builder.getFileMapper() diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java index f6f78e20283..65c262ab5ef 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -68,13 +68,13 @@ public class DoubleGenericColumnPartSerde implements ColumnPartSerde { return (buffer, builder, columnConfig) -> { final CompressedDoublesIndexedSupplier column = CompressedDoublesIndexedSupplier.fromByteBuffer( - buffer, - byteOrder, - builder.getFileMapper() - ); - builder.setType(ValueType.DOUBLE) - .setHasMultipleValues(false) - .setGenericColumn(new DoubleGenericColumnSupplier(column)); + buffer, + byteOrder, + builder.getFileMapper() + ); + builder.setType(ValueType.DOUBLE) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(column)); }; } @@ -89,8 +89,7 @@ public class DoubleGenericColumnPartSerde implements ColumnPartSerde private ByteOrder byteOrder = null; private DoubleColumnSerializer delegate = null; - public - SerializerBuilder withByteOrder(final ByteOrder byteOrder) + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) { this.byteOrder = byteOrder; return this; diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index fe2ca9705fe..d18b796c4d5 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -86,20 +86,21 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde public FloatGenericColumnPartSerde build() { return new FloatGenericColumnPartSerde( - byteOrder, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } + byteOrder, + new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } - @Override - public void write(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException - { - delegate.writeToChannel(channel, fileSmoosher); - } - } + @Override + public void write(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException + { + delegate.writeToChannel(channel, fileSmoosher); + } + } ); } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index e5f943be0bb..e88d131e356 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -86,20 +86,21 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde public LongGenericColumnPartSerde build() { return new LongGenericColumnPartSerde( - byteOrder, new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } + byteOrder, + new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel, smoosher); - } - } + @Override + public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + delegate.writeToChannel(channel, smoosher); + } + } ); } } diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index fb0b18a80c2..c7603797680 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -36,7 +36,6 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.aggregation.AggregationTestHelper; -import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; @@ -73,6 +72,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -156,14 +156,7 @@ public class MultiValuedDimensionTest .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) .setGranularity(Granularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) - .setAggregatorSpecs( - Arrays.asList( - new AggregatorFactory[] - { - new CountAggregatorFactory("count") - } - ) - ) + .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count"))) .build(); Sequence result = helper.runQueryOnSegmentsObjs( @@ -197,17 +190,8 @@ public class MultiValuedDimensionTest .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) .setGranularity(Granularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) - .setAggregatorSpecs( - Arrays.asList( - new AggregatorFactory[] - { - new CountAggregatorFactory("count") - } - ) - ) - .setDimFilter( - new SelectorDimFilter("tags", "t3", null) - ) + .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count"))) + .setDimFilter(new SelectorDimFilter("tags", "t3", null)) .build(); Sequence result = helper.runQueryOnSegmentsObjs( @@ -245,17 +229,8 @@ public class MultiValuedDimensionTest ) ) ) - .setAggregatorSpecs( - Arrays.asList( - new AggregatorFactory[] - { - new CountAggregatorFactory("count") - } - ) - ) - .setDimFilter( - new SelectorDimFilter("tags", "t3", null) - ) + .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count"))) + .setDimFilter(new SelectorDimFilter("tags", "t3", null)) .build(); Sequence result = helper.runQueryOnSegmentsObjs( @@ -286,13 +261,7 @@ public class MultiValuedDimensionTest )) .metric("count") .intervals(QueryRunnerTestHelper.fullOnInterval) - .aggregators( - Arrays.asList( - new AggregatorFactory[] - { - new CountAggregatorFactory("count") - } - )) + .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) .threshold(5) .filters(new SelectorDimFilter("tags", "t3", null)).build(); diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index d3ddb3640ca..f57d61aa570 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -78,9 +78,7 @@ public class FilteredAggregatorTest new SelectorDimFilter("dim", "a", null) ); - FilteredAggregator agg = (FilteredAggregator) factory.factorize( - makeColumnSelector(selector) - ); + FilteredAggregator agg = (FilteredAggregator) factory.factorize(makeColumnSelector(selector)); double expectedFirst = new Float(values[0]).doubleValue(); double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index 1f3117ae3f1..58bb5aaf6d7 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -36,7 +36,6 @@ import io.druid.query.Query; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; -import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; @@ -51,6 +50,7 @@ import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -70,14 +70,7 @@ public class GroupByQueryRunnerFactoryTest .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) .setGranularity(Granularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) - .setAggregatorSpecs( - Arrays.asList( - new AggregatorFactory[] - { - new CountAggregatorFactory("count") - } - ) - ) + .setAggregatorSpecs(Collections.singletonList(new CountAggregatorFactory("count"))) .build(); final QueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 35ba74a9a39..07a9e902eb8 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -2898,13 +2898,11 @@ public class GroupByQueryRunnerTest ); // Now try it with an expression based aggregator. - builder.setLimit(Integer.MAX_VALUE) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new DoubleSumAggregatorFactory("idx", null, "index / 2 + indexMin", TestExprMacroTable.INSTANCE) - ) - ); + List aggregatorSpecs = Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new DoubleSumAggregatorFactory("idx", null, "index / 2 + indexMin", TestExprMacroTable.INSTANCE) + ); + builder.setLimit(Integer.MAX_VALUE).setAggregatorSpecs(aggregatorSpecs); expectedResults = GroupByQueryRunnerTestHelper.createExpectedRows( new String[]{"__time", "alias", "rows", "idx"}, @@ -2930,16 +2928,17 @@ public class GroupByQueryRunnerTest ); // Now try it with an expression virtual column. - builder.setLimit(Integer.MAX_VALUE) - .setVirtualColumns( - new ExpressionVirtualColumn("expr", "index / 2 + indexMin", ValueType.FLOAT, TestExprMacroTable.INSTANCE) - ) - .setAggregatorSpecs( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - new DoubleSumAggregatorFactory("idx", "expr") - ) - ); + ExpressionVirtualColumn expressionVirtualColumn = new ExpressionVirtualColumn( + "expr", + "index / 2 + indexMin", + ValueType.FLOAT, + TestExprMacroTable.INSTANCE + ); + List aggregatorSpecs2 = Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new DoubleSumAggregatorFactory("idx", "expr") + ); + builder.setLimit(Integer.MAX_VALUE).setVirtualColumns(expressionVirtualColumn).setAggregatorSpecs(aggregatorSpecs2); TestHelper.assertExpectedObjects( expectedResults, diff --git a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java index 4ef1aee63c7..a52d2fd4d9d 100644 --- a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java @@ -40,19 +40,23 @@ import static org.junit.Assert.assertTrue; public class HavingSpecTest { - private static final Row ROW = new MapBasedInputRow(0, new ArrayList<>(), ImmutableMap.of("metric", Float.valueOf(10))); + private static final Row ROW = new MapBasedInputRow( + 0, + new ArrayList<>(), + ImmutableMap.of("metric", Float.valueOf(10)) + ); @Test public void testHavingClauseSerde() throws Exception { List havings = Arrays.asList( - new GreaterThanHavingSpec("agg", Double.valueOf(1.3)), - new OrHavingSpec( - Arrays.asList( - new LessThanHavingSpec("lessAgg", Long.valueOf(1L)), - new NotHavingSpec(new EqualToHavingSpec("equalAgg", Double.valueOf(2))) + new GreaterThanHavingSpec("agg", Double.valueOf(1.3)), + new OrHavingSpec( + Arrays.asList( + new LessThanHavingSpec("lessAgg", Long.valueOf(1L)), + new NotHavingSpec(new EqualToHavingSpec("equalAgg", Double.valueOf(2))) + ) ) - ) ); HavingSpec andHavingSpec = new AndHavingSpec(havings); @@ -97,7 +101,7 @@ public class HavingSpecTest "value", 1.3 ); ObjectMapper mapper = new DefaultObjectMapper(); - HavingSpec spec = mapper.convertValue (greaterMap, HavingSpec.class); + HavingSpec spec = mapper.convertValue(greaterMap, HavingSpec.class); } @@ -161,6 +165,7 @@ public class HavingSpecTest private final AtomicInteger counter; private final boolean value; + private CountingHavingSpec(AtomicInteger counter, boolean value) { this.counter = counter; @@ -180,10 +185,10 @@ public class HavingSpecTest { AtomicInteger counter = new AtomicInteger(0); AndHavingSpec spec = new AndHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, false) + (HavingSpec) new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, false) )); spec.eval(ROW); @@ -196,10 +201,10 @@ public class HavingSpecTest { AtomicInteger counter = new AtomicInteger(0); AndHavingSpec spec = new AndHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true) + (HavingSpec) new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true) )); spec.eval(ROW); @@ -208,10 +213,10 @@ public class HavingSpecTest counter.set(0); spec = new AndHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true) + (HavingSpec) new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true) )); spec.eval(ROW); @@ -224,10 +229,10 @@ public class HavingSpecTest { AtomicInteger counter = new AtomicInteger(0); OrHavingSpec spec = new OrHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, true), - new CountingHavingSpec(counter, false) + (HavingSpec) new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, true), + new CountingHavingSpec(counter, false) )); spec.eval(ROW); @@ -240,10 +245,10 @@ public class HavingSpecTest { AtomicInteger counter = new AtomicInteger(0); OrHavingSpec spec = new OrHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, false) + (HavingSpec) new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, false) )); spec.eval(ROW); @@ -252,10 +257,10 @@ public class HavingSpecTest counter.set(0); spec = new OrHavingSpec(ImmutableList.of( - (HavingSpec) new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, false), - new CountingHavingSpec(counter, true) + (HavingSpec) new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, false), + new CountingHavingSpec(counter, true) )); spec.eval(ROW); diff --git a/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java index 143deb9028f..1d1408f91c3 100644 --- a/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java +++ b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSequenceTest.java @@ -55,27 +55,27 @@ public class TopNSequenceTest @Parameterized.Parameters public static Collection makeTestData() { - Object[][] data = new Object[][] { - {ASC, RAW_ASC, RAW_ASC.size() - 2}, - {ASC, RAW_ASC, RAW_ASC.size()}, - {ASC, RAW_ASC, RAW_ASC.size() + 2}, - {ASC, RAW_ASC, 0}, - {ASC, SINGLE, 0}, - {ASC, SINGLE, 1}, - {ASC, SINGLE, 2}, - {ASC, SINGLE, 3}, - {ASC, EMPTY, 0}, - {ASC, EMPTY, 1}, - {DESC, RAW_DESC, RAW_DESC.size() - 2}, - {DESC, RAW_DESC, RAW_DESC.size()}, - {DESC, RAW_DESC, RAW_DESC.size() + 2}, - {DESC, RAW_DESC, 0}, - {DESC, RAW_DESC, 0}, - {DESC, SINGLE, 1}, - {DESC, SINGLE, 2}, - {DESC, SINGLE, 3}, - {DESC, EMPTY, 0}, - {DESC, EMPTY, 1}, + Object[][] data = new Object[][]{ + {ASC, RAW_ASC, RAW_ASC.size() - 2}, + {ASC, RAW_ASC, RAW_ASC.size()}, + {ASC, RAW_ASC, RAW_ASC.size() + 2}, + {ASC, RAW_ASC, 0}, + {ASC, SINGLE, 0}, + {ASC, SINGLE, 1}, + {ASC, SINGLE, 2}, + {ASC, SINGLE, 3}, + {ASC, EMPTY, 0}, + {ASC, EMPTY, 1}, + {DESC, RAW_DESC, RAW_DESC.size() - 2}, + {DESC, RAW_DESC, RAW_DESC.size()}, + {DESC, RAW_DESC, RAW_DESC.size() + 2}, + {DESC, RAW_DESC, 0}, + {DESC, RAW_DESC, 0}, + {DESC, SINGLE, 1}, + {DESC, SINGLE, 2}, + {DESC, SINGLE, 3}, + {DESC, EMPTY, 0}, + {DESC, EMPTY, 1} }; return Arrays.asList(data); diff --git a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java index 2e74bbf2385..f1332e21ec5 100644 --- a/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/lookup/RegisteredLookupExtractionFnTest.java @@ -225,39 +225,40 @@ public class RegisteredLookupExtractionFnTest { EasyMock.expect(manager.get(EasyMock.eq(LOOKUP_NAME))).andReturn( new LookupExtractorFactoryContainer( - "v0", new LookupExtractorFactory() - { - @Override - public boolean start() - { - return false; - } + "v0", + new LookupExtractorFactory() + { + @Override + public boolean start() + { + return false; + } - @Override - public boolean replaces(@Nullable LookupExtractorFactory other) - { - return false; - } + @Override + public boolean replaces(@Nullable LookupExtractorFactory other) + { + return false; + } - @Override - public boolean close() - { - return false; - } + @Override + public boolean close() + { + return false; + } - @Nullable - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - return null; - } + @Nullable + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + return null; + } - @Override - public LookupExtractor get() - { - return LOOKUP_EXTRACTOR; - } - } + @Override + public LookupExtractor get() + { + return LOOKUP_EXTRACTOR; + } + } ) ).anyTimes(); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java index 1742fa41107..3acf7e759ab 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java @@ -112,7 +112,7 @@ public class TimeBoundaryQueryQueryToolChestTest ); for (int i = 0; i < segments.size(); i++) { - Assert.assertEquals(segments.get(i).getInterval(), expected.get(i).getInterval()); + Assert.assertEquals(segments.get(i).getInterval(), expected.get(i).getInterval()); } } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 8d7b57056fb..737acd55ab9 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -667,14 +667,14 @@ public class TimeseriesQueryRunnerTest new Interval(DateTimes.of("2011-04-14T01"), DateTimes.of("2011-04-15")) ); for (Interval interval : iterable) { - lotsOfZeroes.add( - new Result<>( - interval.getStart(), - new TimeseriesResultValue( - ImmutableMap.of("rows", 0L, "idx", 0L) - ) - ) - ); + lotsOfZeroes.add( + new Result<>( + interval.getStart(), + new TimeseriesResultValue( + ImmutableMap.of("rows", 0L, "idx", 0L) + ) + ) + ); } List> expectedResults1 = Lists.newArrayList( diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 998a4334919..5c026ca2443 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -21,7 +21,6 @@ package io.druid.query.topn; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -162,14 +161,7 @@ public class TopNQueryRunnerTest new TopNQueryRunnerFactory( new StupidPool( "TopNQueryRunnerFactory-bufferPool", - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(20000); - } - } + () -> ByteBuffer.allocate(20000) ), new TopNQueryQueryToolChest( new TopNQueryConfig(), diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 4d1804a3568..1a503675d85 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -19,7 +19,6 @@ package io.druid.query.topn; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -78,14 +77,7 @@ public class TopNUnionQueryTest new TopNQueryRunnerFactory( new StupidPool( "TopNQueryRunnerFactory-bufferPool", - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(2000); - } - } + () -> ByteBuffer.allocate(2000) ), new TopNQueryQueryToolChest( new TopNQueryConfig(), diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java index d4470ee9dcf..429b603c53a 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -203,9 +203,8 @@ public class IndexIOTest return new Object[]{ maplist1, maplist2, - filterNullValues(maplist1).equals(filterNullValues(maplist2)) - ? null - : SegmentValidationException.class + filterNullValues(maplist1).equals(filterNullValues(maplist2)) ? + null : SegmentValidationException.class }; } diff --git a/processing/src/test/java/io/druid/segment/MetadataTest.java b/processing/src/test/java/io/druid/segment/MetadataTest.java index 763bf00dc4d..6ae55daadab 100644 --- a/processing/src/test/java/io/druid/segment/MetadataTest.java +++ b/processing/src/test/java/io/druid/segment/MetadataTest.java @@ -45,10 +45,9 @@ public class MetadataTest Metadata metadata = new Metadata(); metadata.put("k", "v"); - AggregatorFactory[] aggregators = new AggregatorFactory[] - { - new LongSumAggregatorFactory("out", "in") - }; + AggregatorFactory[] aggregators = new AggregatorFactory[] { + new LongSumAggregatorFactory("out", "in") + }; metadata.setAggregators(aggregators); metadata.setQueryGranularity(Granularities.ALL); metadata.setRollup(Boolean.FALSE); diff --git a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java index bbcc3697ca3..788c465e038 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -646,25 +646,25 @@ public class IncrementalIndexTest Map context = new HashMap(); Sequence> sequence = runner.run(QueryPlus.wrap(query), context); - for (Double result : - sequence.accumulate( - new Double[0], new Accumulator>() - { - @Override - public Double[] accumulate( - Double[] accumulated, Result in - ) - { - if (currentlyRunning.get() > 0) { - concurrentlyRan.incrementAndGet(); - } - queriesAccumualted.incrementAndGet(); - return Lists.asList(in.getValue().getDoubleMetric("doubleSumResult0"), accumulated) - .toArray(new Double[accumulated.length + 1]); - } + Double[] results = sequence.accumulate( + new Double[0], + new Accumulator>() + { + @Override + public Double[] accumulate( + Double[] accumulated, Result in + ) + { + if (currentlyRunning.get() > 0) { + concurrentlyRan.incrementAndGet(); } - ) - ) { + queriesAccumualted.incrementAndGet(); + return Lists.asList(in.getValue().getDoubleMetric("doubleSumResult0"), accumulated) + .toArray(new Double[accumulated.length + 1]); + } + } + ); + for (Double result : results) { final Integer maxValueExpected = someoneRan.get() + concurrentThreads; if (maxValueExpected > 0) { // Eventually consistent, but should be somewhere in that range diff --git a/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java index ee46c55bde6..215a4d12f05 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java @@ -79,7 +79,7 @@ public class VSizeIndexedTest Assert.assertEquals(ints.length, vSizeIndexedInts.size()); Assert.assertEquals(1, vSizeIndexedInts.getNumBytes()); for (int j = 0; j < ints.length; j++) { - Assert.assertEquals(ints[j], vSizeIndexedInts.get(j)); + Assert.assertEquals(ints[j], vSizeIndexedInts.get(j)); } } } diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index f6ecc26c13c..908f4896c53 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -393,12 +393,11 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark .aggregators(queryAggregatorFactories) .build(); Map context = new HashMap(); - for (Result result : - Sequences.toList( - runner.run(QueryPlus.wrap(query), context), - new LinkedList>() - ) - ) { + LinkedList> results = Sequences.toList( + runner.run(QueryPlus.wrap(query), context), + new LinkedList>() + ); + for (Result result : results) { if (someoneRan.get()) { Assert.assertTrue(result.getValue().getDoubleMetric("doubleSumResult0") > 0); } diff --git a/server/src/main/java/io/druid/guice/FireDepartmentsProvider.java b/server/src/main/java/io/druid/guice/FireDepartmentsProvider.java index 59582ccbd92..7e6b0a249db 100644 --- a/server/src/main/java/io/druid/guice/FireDepartmentsProvider.java +++ b/server/src/main/java/io/druid/guice/FireDepartmentsProvider.java @@ -43,11 +43,7 @@ public class FireDepartmentsProvider implements Provider> { try { this.fireDepartments.addAll( - (List) jsonMapper.readValue( - config.getSpecFile(), new TypeReference>() - { - } - ) + jsonMapper.readValue(config.getSpecFile(), new TypeReference>() {}) ); } catch (Exception e) { diff --git a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java index aa38e663ad3..b6e52bd551d 100644 --- a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java @@ -98,10 +98,10 @@ public class LocalDataSegmentPusher implements DataSegmentPusher final long size = compressSegment(dataSegmentFile, tmpOutDir); final DataSegment dataSegment = createDescriptorFile( - segment.withLoadSpec(makeLoadSpec(new File(outDir, "index.zip").toURI())) - .withSize(size) - .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), - tmpOutDir + segment.withLoadSpec(makeLoadSpec(new File(outDir, "index.zip").toURI())) + .withSize(size) + .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), + tmpOutDir ); // moving the temporary directory to the final destination, once success the potentially concurrent push operations diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java index c3e8678ba30..56242ba8777 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorPlumber.java @@ -350,41 +350,38 @@ public class AppenderatorPlumber implements Plumber ) ); - ScheduledExecutors - .scheduleAtFixedRate( - scheduledExecutor, - new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ), - new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)), - new ThreadRenamingCallable( - StringUtils.format( - "%s-overseer-%d", - schema.getDataSource(), - config.getShardSpec().getPartitionNum() - ) - ) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } - - mergeAndPush(); - - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } + String threadName = StringUtils.format( + "%s-overseer-%d", + schema.getDataSource(), + config.getShardSpec().getPartitionNum() + ); + ThreadRenamingCallable threadRenamingCallable = + new ThreadRenamingCallable(threadName) + { + @Override + public ScheduledExecutors.Signal doCall() + { + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; } - ); + + mergeAndPush(); + + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; + } else { + return ScheduledExecutors.Signal.REPEAT; + } + } + }; + Duration initialDelay = new Duration( + System.currentTimeMillis(), + segmentGranularity.increment(truncatedNow).getMillis() + windowMillis + ); + Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); + ScheduledExecutors.scheduleAtFixedRate(scheduledExecutor, initialDelay, rate, threadRenamingCallable); } private void mergeAndPush() diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/io/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java index 9ff9c8edf00..6be9b1b355f 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java @@ -39,8 +39,7 @@ import io.druid.timeline.partition.ShardSpec; import java.io.File; import java.util.concurrent.ExecutorService; -public class -DefaultRealtimeAppenderatorFactory implements AppenderatorFactory +public class DefaultRealtimeAppenderatorFactory implements AppenderatorFactory { private final ServiceEmitter emitter; private final QueryRunnerFactoryConglomerate conglomerate; diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java index 132f669725c..813a638739c 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java @@ -149,10 +149,10 @@ class WikipediaIrcDecoder implements IrcDecoder File tmpFile = File.createTempFile("druid", "geo"); FileUtils.copyInputStreamToFile( - new GZIPInputStream( - new URL("http://geolite.maxmind.com/download/geoip/database/GeoLite2-City.mmdb.gz").openStream() - ), - tmpFile + new GZIPInputStream( + new URL("http://geolite.maxmind.com/download/geoip/database/GeoLite2-City.mmdb.gz").openStream() + ), + tmpFile ); if (!tmpFile.renameTo(geoDb)) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 3938d017a92..5a1b76a25b0 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -157,56 +157,53 @@ public class FlushingPlumber extends RealtimePlumber ) ); - ScheduledExecutors - .scheduleAtFixedRate( - flushScheduledExec, - new Duration( - System.currentTimeMillis(), - schema.getGranularitySpec().getSegmentGranularity().increment(truncatedNow).getMillis() + windowMillis - ), - new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)), - new ThreadRenamingCallable( - StringUtils.format( - "%s-flusher-%d", - getSchema().getDataSource(), - getConfig().getShardSpec().getPartitionNum() - ) - ) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping flusher thread"); - return ScheduledExecutors.Signal.STOP; - } + String threadName = StringUtils.format( + "%s-flusher-%d", + getSchema().getDataSource(), + getConfig().getShardSpec().getPartitionNum() + ); + ThreadRenamingCallable threadRenamingCallable = + new ThreadRenamingCallable(threadName) + { + @Override + public ScheduledExecutors.Signal doCall() + { + if (stopped) { + log.info("Stopping flusher thread"); + return ScheduledExecutors.Signal.STOP; + } - long minTimestamp = segmentGranularity.bucketStart( - getRejectionPolicy().getCurrMaxTime().minus(windowMillis) - ).getMillis(); + long minTimestamp = segmentGranularity.bucketStart( + getRejectionPolicy().getCurrMaxTime().minus(windowMillis) + ).getMillis(); - List> sinksToPush = Lists.newArrayList(); - for (Map.Entry entry : getSinks().entrySet()) { - final Long intervalStart = entry.getKey(); - if (intervalStart < minTimestamp) { - log.info("Adding entry[%s] to flush.", entry); - sinksToPush.add(entry); - } - } - - for (final Map.Entry entry : sinksToPush) { - flushAfterDuration(entry.getKey(), entry.getValue()); - } - - if (stopped) { - log.info("Stopping flusher thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } + List> sinksToPush = Lists.newArrayList(); + for (Map.Entry entry : getSinks().entrySet()) { + final Long intervalStart = entry.getKey(); + if (intervalStart < minTimestamp) { + log.info("Adding entry[%s] to flush.", entry); + sinksToPush.add(entry); } } - ); + + for (final Map.Entry entry : sinksToPush) { + flushAfterDuration(entry.getKey(), entry.getValue()); + } + + if (stopped) { + log.info("Stopping flusher thread"); + return ScheduledExecutors.Signal.STOP; + } else { + return ScheduledExecutors.Signal.REPEAT; + } + } + }; + Duration initialDelay = new Duration( + System.currentTimeMillis(), + schema.getGranularitySpec().getSegmentGranularity().increment(truncatedNow).getMillis() + windowMillis + ); + Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); + ScheduledExecutors.scheduleAtFixedRate(flushScheduledExec, initialDelay, rate, threadRenamingCallable); } @Override diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 40fe89057ed..5239d591ccd 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -769,41 +769,38 @@ public class RealtimePlumber implements Plumber ) ); - ScheduledExecutors - .scheduleAtFixedRate( - scheduledExecutor, - new Duration( - System.currentTimeMillis(), - segmentGranularity.increment(truncatedNow).getMillis() + windowMillis - ), - new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)), - new ThreadRenamingCallable( - StringUtils.format( - "%s-overseer-%d", - schema.getDataSource(), - config.getShardSpec().getPartitionNum() - ) - ) - { - @Override - public ScheduledExecutors.Signal doCall() - { - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } - - mergeAndPush(); - - if (stopped) { - log.info("Stopping merge-n-push overseer thread"); - return ScheduledExecutors.Signal.STOP; - } else { - return ScheduledExecutors.Signal.REPEAT; - } - } + String threadName = StringUtils.format( + "%s-overseer-%d", + schema.getDataSource(), + config.getShardSpec().getPartitionNum() + ); + ThreadRenamingCallable threadRenamingCallable = + new ThreadRenamingCallable(threadName) + { + @Override + public ScheduledExecutors.Signal doCall() + { + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; } - ); + + mergeAndPush(); + + if (stopped) { + log.info("Stopping merge-n-push overseer thread"); + return ScheduledExecutors.Signal.STOP; + } else { + return ScheduledExecutors.Signal.REPEAT; + } + } + }; + Duration initialDelay = new Duration( + System.currentTimeMillis(), + segmentGranularity.increment(truncatedNow).getMillis() + windowMillis + ); + Duration rate = new Duration(truncatedNow, segmentGranularity.increment(truncatedNow)); + ScheduledExecutors.scheduleAtFixedRate(scheduledExecutor, initialDelay, rate, threadRenamingCallable); } private void mergeAndPush() diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 39e16044ffb..f8f7d67576e 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -190,21 +190,20 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet implements Qu // to keep the code simple, the proxy servlet will also send a request to one of the default brokers if (!server.getHost().equals(defaultServer.getHost())) { // issue async requests + Response.CompleteListener completeListener = result -> { + if (result.isFailed()) { + log.warn( + result.getFailure(), + "Failed to forward cancellation request to [%s]", + server.getHost() + ); + } + }; broadcastClient .newRequest(rewriteURI(request, server.getScheme(), server.getHost())) .method(HttpMethod.DELETE) .timeout(CANCELLATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS) - .send( - result -> { - if (result.isFailed()) { - log.warn( - result.getFailure(), - "Failed to forward cancellation request to [%s]", - server.getHost() - ); - } - } - ); + .send(completeListener); } interruptedQueryCount.incrementAndGet(); } diff --git a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java index c20cb13a1f4..9bb94593f43 100644 --- a/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java +++ b/server/src/main/java/io/druid/server/coordination/broker/DruidBroker.java @@ -47,7 +47,7 @@ public class DruidBroker final FilteredServerInventoryView serverInventoryView, final @Self DruidNode self, final ServiceAnnouncer serviceAnnouncer - ) + ) { this.self = self; this.serviceAnnouncer = serviceAnnouncer; diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 3e5d2c68fdd..653a62af16a 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -246,14 +246,12 @@ public class DruidCoordinator ((LoadRule) rule) .getTieredReplicants() - .forEach( - (final String tier, final Integer ruleReplicants) -> { - int currentReplicants = segmentReplicantLookup.getTotalReplicants(segment.getIdentifier(), tier); - retVal - .computeIfAbsent(tier, ignored -> new Object2LongOpenHashMap<>()) - .addTo(segment.getDataSource(), Math.max(ruleReplicants - currentReplicants, 0)); - } - ); + .forEach((final String tier, final Integer ruleReplicants) -> { + int currentReplicants = segmentReplicantLookup.getTotalReplicants(segment.getIdentifier(), tier); + retVal + .computeIfAbsent(tier, ignored -> new Object2LongOpenHashMap<>()) + .addTo(segment.getDataSource(), Math.max(ruleReplicants - currentReplicants, 0)); + }); } } diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index aff7093cf84..81eab183bdc 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -275,11 +275,11 @@ public class LoadQueuePeon public void process(WatchedEvent watchedEvent) throws Exception { switch (watchedEvent.getType()) { - case NodeDeleted: - entryRemoved(watchedEvent.getPath()); - break; - default: - // do nothing + case NodeDeleted: + entryRemoved(watchedEvent.getPath()); + break; + default: + // do nothing } } } diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java index 4ab3870c0f1..3820dfd0ee1 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -199,34 +199,32 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper // Emit coordinator metrics params .getLoadManagementPeons() - .forEach( - (final String serverName, final LoadQueuePeon queuePeon) -> { - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/size", queuePeon.getLoadQueueSize() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/loadQueue/count", queuePeon.getSegmentsToLoad().size() - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.SERVER, serverName).build( - "segment/dropQueue/count", queuePeon.getSegmentsToDrop().size() - ) - ); - } - ); + .forEach((final String serverName, final LoadQueuePeon queuePeon) -> { + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/size", queuePeon.getLoadQueueSize() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/loadQueue/count", queuePeon.getSegmentsToLoad().size() + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.SERVER, serverName).build( + "segment/dropQueue/count", queuePeon.getSegmentsToDrop().size() + ) + ); + }); coordinator.getSegmentAvailability().object2LongEntrySet().forEach( (final Object2LongMap.Entry entry) -> { @@ -266,23 +264,21 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper allSegments .collect(Collectors.groupingBy(DataSegment::getDataSource)) - .forEach( - (final String name, final List segments) -> { - final long size = segments.stream().mapToLong(DataSegment::getSize).sum(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, name).build( - "segment/size", size - ) - ); - emitter.emit( - new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, name).build( - "segment/count", segments.size() - ) - ); - } - ); + .forEach((final String name, final List segments) -> { + final long size = segments.stream().mapToLong(DataSegment::getSize).sum(); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, name).build( + "segment/size", size + ) + ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, name).build( + "segment/count", segments.size() + ) + ); + }); return params; } diff --git a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java index 879876ee6d5..5fc7fb2340e 100644 --- a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java @@ -25,29 +25,22 @@ import org.apache.curator.utils.ZKPaths; public class ZkPathsConfig { @JsonProperty - private - String base = "druid"; + private String base = "druid"; @JsonProperty - private - String propertiesPath; + private String propertiesPath; @JsonProperty - private - String announcementsPath; - @JsonProperty @Deprecated - private - String servedSegmentsPath; + private String announcementsPath; @JsonProperty - private - String liveSegmentsPath; + @Deprecated + private String servedSegmentsPath; @JsonProperty - private - String coordinatorPath; + private String liveSegmentsPath; @JsonProperty - private - String loadQueuePath; + private String coordinatorPath; @JsonProperty - private - String connectorPath; + private String loadQueuePath; + @JsonProperty + private String connectorPath; public String getBase() { diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index f45018f8a40..88cd3697efa 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -2472,8 +2472,7 @@ public class CachingClusteredClientTest ); } - private Iterable> makeTimeResults - (Object... objects) + private Iterable> makeTimeResults(Object... objects) { if (objects.length % 3 != 0) { throw new ISE("makeTimeResults must be passed arguments in groups of 3, got[%d]", objects.length); @@ -2501,8 +2500,7 @@ public class CachingClusteredClientTest return retVal; } - private Iterable> makeBySegmentTimeResults - (Object... objects) + private Iterable> makeBySegmentTimeResults(Object... objects) { if (objects.length % 5 != 0) { throw new ISE("makeTimeResults must be passed arguments in groups of 5, got[%d]", objects.length); @@ -2532,8 +2530,7 @@ public class CachingClusteredClientTest return retVal; } - private Iterable> makeRenamedTimeResults - (Object... objects) + private Iterable> makeRenamedTimeResults(Object... objects) { if (objects.length % 3 != 0) { throw new ISE("makeTimeResults must be passed arguments in groups of 3, got[%d]", objects.length); @@ -2557,8 +2554,7 @@ public class CachingClusteredClientTest return retVal; } - private Iterable> makeTopNResultsWithoutRename - (Object... objects) + private Iterable> makeTopNResultsWithoutRename(Object... objects) { return makeTopNResults( Lists.newArrayList( @@ -2574,8 +2570,7 @@ public class CachingClusteredClientTest ); } - private Iterable> makeTopNResults - (List names, Object... objects) + private Iterable> makeTopNResults(List names, Object... objects) { Preconditions.checkArgument(names.size() == 7); List> retVal = Lists.newArrayList(); @@ -2611,8 +2606,7 @@ public class CachingClusteredClientTest return retVal; } - private Iterable> makeRenamedTopNResults - (Object... objects) + private Iterable> makeRenamedTopNResults(Object... objects) { return makeTopNResults( Lists.newArrayList( @@ -2628,8 +2622,7 @@ public class CachingClusteredClientTest ); } - private Iterable> makeSearchResults - (String dim, Object... objects) + private Iterable> makeSearchResults(String dim, Object... objects) { List> retVal = Lists.newArrayList(); int index = 0; diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 37c9734c528..0ba202eb009 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -391,8 +391,7 @@ public class CachingQueryRunnerTest Assert.assertEquals(expectedResults.toString(), results.toString()); } - private List makeTopNResults - (boolean cachedResults, Object... objects) + private List makeTopNResults(boolean cachedResults, Object... objects) { List retVal = Lists.newArrayList(); int index = 0; diff --git a/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java b/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java index 41f1f6df214..9eb8899d11c 100644 --- a/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java +++ b/server/src/test/java/io/druid/client/selector/TierSelectorStrategyTest.java @@ -97,14 +97,15 @@ public class TierSelectorStrategyTest testTierSelectorStrategy( new CustomTierSelectorStrategy( - new ConnectionCountServerSelectorStrategy(), new CustomTierSelectorStrategyConfig() - { - @Override - public List getPriorities() - { - return Arrays.asList(2, 0, -1, 1); - } - } + new ConnectionCountServerSelectorStrategy(), + new CustomTierSelectorStrategyConfig() + { + @Override + public List getPriorities() + { + return Arrays.asList(2, 0, -1, 1); + } + } ), mediumPriority, lowPriority, highPriority ); diff --git a/server/src/test/java/io/druid/curator/CuratorTestBase.java b/server/src/test/java/io/druid/curator/CuratorTestBase.java index d50f65cb271..6ad32306172 100644 --- a/server/src/test/java/io/druid/curator/CuratorTestBase.java +++ b/server/src/test/java/io/druid/curator/CuratorTestBase.java @@ -64,13 +64,11 @@ public class CuratorTestBase final String announcementsPath = zkPathsConfig.getAnnouncementsPath(); final String inventoryPath = zkPathsConfig.getLiveSegmentsPath(); + String zkPath = ZKPaths.makePath(announcementsPath, server.getHost()); try { curator.create() .creatingParentsIfNeeded() - .forPath( - ZKPaths.makePath(announcementsPath, server.getHost()), - jsonMapper.writeValueAsBytes(server.getMetadata()) - ); + .forPath(zkPath, jsonMapper.writeValueAsBytes(server.getMetadata())); curator.create() .creatingParentsIfNeeded() .forPath(ZKPaths.makePath(inventoryPath, server.getHost())); @@ -84,10 +82,7 @@ public class CuratorTestBase */ try { curator.setData() - .forPath( - ZKPaths.makePath(announcementsPath, server.getHost()), - jsonMapper.writeValueAsBytes(server.getMetadata()) - ); + .forPath(zkPath, jsonMapper.writeValueAsBytes(server.getMetadata())); curator.setData() .forPath(ZKPaths.makePath(inventoryPath, server.getHost())); } @@ -116,20 +111,12 @@ public class CuratorTestBase curator.create() .compressed() .withMode(CreateMode.EPHEMERAL) - .forPath( - segmentAnnouncementPath, - jsonMapper.writeValueAsBytes( - ImmutableSet.of(segment) - ) - ); + .forPath(segmentAnnouncementPath, jsonMapper.writeValueAsBytes(ImmutableSet.of(segment))); } catch (KeeperException.NodeExistsException e) { try { curator.setData() - .forPath( - segmentAnnouncementPath, - jsonMapper.writeValueAsBytes(ImmutableSet.of(segment)) - ); + .forPath(segmentAnnouncementPath, jsonMapper.writeValueAsBytes(ImmutableSet.of(segment))); } catch (Exception e1) { Throwables.propagate(e1); diff --git a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java index 6c94da7f896..b88ca449b38 100644 --- a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java +++ b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java @@ -159,8 +159,7 @@ public class LookupDimensionSpecTest @Parameters public void testApply(DimensionSpec dimensionSpec, Map map) { - for (Map.Entry entry : map.entrySet() - ) { + for (Map.Entry entry : map.entrySet()) { Assert.assertEquals(Strings.emptyToNull(entry.getValue()), dimensionSpec.getExtractionFn().apply(entry.getKey())); } } diff --git a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java index 67dd673677e..65fad2a71a6 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupIntrospectionResourceImplTest.java @@ -39,7 +39,7 @@ import javax.ws.rs.ext.Provider; public class LookupIntrospectionResourceImplTest extends JerseyTest { - static LookupReferencesManager lookupReferencesManager = EasyMock.createMock(LookupReferencesManager.class); + static LookupReferencesManager lookupReferencesManager = EasyMock.createMock(LookupReferencesManager.class); @Override diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 90b1e66f4b2..f71739b50a1 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -464,14 +464,18 @@ public class RealtimeManagerTest realtimeManager3.start(); - while (realtimeManager3.getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ) - ) { + while (true) { + boolean notAllStarted = realtimeManager3 + .getFireChiefs("testing").values().stream() + .anyMatch( + fireChief -> { + final Plumber plumber = fireChief.getPlumber(); + return plumber == null || !((TestPlumber) plumber).isStartedJob(); + } + ); + if (!notAllStarted) { + break; + } Thread.sleep(10); } @@ -533,14 +537,18 @@ public class RealtimeManagerTest realtimeManager3.start(); - while (realtimeManager3.getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ) - ) { + while (true) { + boolean notAllStarted = realtimeManager3 + .getFireChiefs("testing").values().stream() + .anyMatch( + fireChief -> { + final Plumber plumber = fireChief.getPlumber(); + return plumber == null || !((TestPlumber) plumber).isStartedJob(); + } + ); + if (!notAllStarted) { + break; + } Thread.sleep(10); } @@ -641,14 +649,18 @@ public class RealtimeManagerTest realtimeManager3.start(); - while (realtimeManager3.getFireChiefs("testing").values().stream() - .anyMatch( - fireChief -> { - final Plumber plumber = fireChief.getPlumber(); - return plumber == null || !((TestPlumber) plumber).isStartedJob(); - } - ) - ) { + while (true) { + boolean notAllStarted = realtimeManager3 + .getFireChiefs("testing").values().stream() + .anyMatch( + fireChief -> { + final Plumber plumber = fireChief.getPlumber(); + return plumber == null || !((TestPlumber) plumber).isStartedJob(); + } + ); + if (!notAllStarted) { + break; + } Thread.sleep(10); } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index f2ebc839d64..52a5aa9a431 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -63,45 +63,46 @@ public class DefaultOfflineAppenderatorFactoryTest { Injector injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), - ImmutableList.of(new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/tool"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(9999); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - binder.bind(DruidProcessingConfig.class).toInstance( - new DruidProcessingConfig() - { - @Override - public String getFormatString() - { - return "processing-%s"; - } + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/tool"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(9999); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + binder.bind(DruidProcessingConfig.class).toInstance( + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return "processing-%s"; + } - @Override - public int intermediateComputeSizeBytes() - { - return 100 * 1024 * 1024; - } + @Override + public int intermediateComputeSizeBytes() + { + return 100 * 1024 * 1024; + } - @Override - public int getNumThreads() - { - return 1; - } + @Override + public int getNumThreads() + { + return 1; + } - @Override - public int columnCacheSizeBytes() - { - return 25 * 1024 * 1024; - } - } - ); - binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); - } - } + @Override + public int columnCacheSizeBytes() + { + return 25 * 1024 * 1024; + } + } + ); + binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); + } + } ) ); ObjectMapper objectMapper = injector.getInstance(ObjectMapper.class); diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 8e17dcce08c..a2329cb44f7 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -281,7 +281,7 @@ public class EventReceiverFirehoseTest firehose.shutdown(DateTimes.nowUtc().plusMillis(100).toString(), req); while (!firehose.isClosed()) { - Thread.sleep(50); + Thread.sleep(50); } } } diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index a9a9c1ab48e..5f1586a0d67 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -255,18 +255,15 @@ public class RealtimePlumberSchoolTest private void testPersist(final Object commitMetadata) throws Exception { - plumber.getSinks() - .put( - 0L, - new Sink( - Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), - schema, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.isReportParseExceptions() - ) - ); + Sink sink = new Sink( + Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), + schema, + tuningConfig.getShardSpec(), + DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + ); + plumber.getSinks().put(0L, sink); Assert.assertNull(plumber.startJob()); final InputRow row = EasyMock.createNiceMock(InputRow.class); @@ -302,18 +299,15 @@ public class RealtimePlumberSchoolTest @Test(timeout = 60000) public void testPersistFails() throws Exception { - plumber.getSinks() - .put( - 0L, - new Sink( - Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), - schema, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.isReportParseExceptions() - ) - ); + Sink sink = new Sink( + Intervals.utc(0, TimeUnit.HOURS.toMillis(1)), + schema, + tuningConfig.getShardSpec(), + DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + ); + plumber.getSinks().put(0L, sink); plumber.startJob(); final InputRow row = EasyMock.createNiceMock(InputRow.class); EasyMock.expect(row.getTimestampFromEpoch()).andReturn(0L); @@ -359,18 +353,15 @@ public class RealtimePlumberSchoolTest Interval testInterval = new Interval(DateTimes.of("1970-01-01"), DateTimes.of("1971-01-01")); RealtimePlumber plumber2 = (RealtimePlumber) realtimePlumberSchool.findPlumber(schema2, tuningConfig, metrics); - plumber2.getSinks() - .put( - 0L, - new Sink( - testInterval, - schema2, - tuningConfig.getShardSpec(), - DateTimes.of("2014-12-01T12:34:56.789").toString(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.isReportParseExceptions() - ) - ); + Sink sink = new Sink( + testInterval, + schema2, + tuningConfig.getShardSpec(), + DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.isReportParseExceptions() + ); + plumber2.getSinks().put(0L, sink); Assert.assertNull(plumber2.startJob()); final CountDownLatch doneSignal = new CountDownLatch(1); final Committer committer = new Committer() diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 736c13bf748..fc2eb94539a 100644 --- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -317,8 +317,8 @@ public class BatchDataSegmentAnnouncerTest new SegmentChangeRequestHistory.Counter(-1, -1) ).get(); Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); - Assert.assertEquals(testSegments.size(), snapshot.getCounter().getCounter()); - } + Assert.assertEquals(testSegments.size(), snapshot.getCounter().getCounter()); + } segmentAnnouncer.unannounceSegments(testSegments); diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index 0946c362b15..d6a8b000b98 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -136,14 +136,20 @@ public class DruidCoordinatorTest extends CuratorTestBase false, new Duration("PT0s") ); - pathChildrenCache = new PathChildrenCache(curator, LOADPATH, true, true, Execs.singleThreaded("coordinator_test_path_children_cache-%d")); + pathChildrenCache = new PathChildrenCache( + curator, + LOADPATH, + true, + true, + Execs.singleThreaded("coordinator_test_path_children_cache-%d") + ); loadQueuePeon = new LoadQueuePeon( - curator, - LOADPATH, - objectMapper, - Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_scheduled-%d"), - Execs.singleThreaded("coordinator_test_load_queue_peon-%d"), - druidCoordinatorConfig + curator, + LOADPATH, + objectMapper, + Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_scheduled-%d"), + Execs.singleThreaded("coordinator_test_load_queue_peon-%d"), + druidCoordinatorConfig ); loadQueuePeon.start(); druidNode = new DruidNode("hey", "what", 1234, null, new ServerConfig()); @@ -178,7 +184,8 @@ public class DruidCoordinatorTest extends CuratorTestBase scheduledExecutorFactory, null, null, - new NoopServiceAnnouncer() { + new NoopServiceAnnouncer() + { @Override public void announce(DruidNode node) { @@ -306,7 +313,17 @@ public class DruidCoordinatorTest extends CuratorTestBase DruidDataSource[] druidDataSources = { new DruidDataSource(dataSource, Collections.emptyMap()) }; - final DataSegment dataSegment = new DataSegment(dataSource, Intervals.of("2010-01-01/P1D"), "v1", null, null, null, null, 0x9, 0); + final DataSegment dataSegment = new DataSegment( + dataSource, + Intervals.of("2010-01-01/P1D"), + "v1", + null, + null, + null, + null, + 0x9, + 0 + ); druidDataSources[0].addSegment("0", dataSegment); EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes(); @@ -338,25 +355,25 @@ public class DruidCoordinatorTest extends CuratorTestBase final CountDownLatch assignSegmentLatch = new CountDownLatch(1); pathChildrenCache.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent( - CuratorFramework curatorFramework, PathChildrenCacheEvent pathChildrenCacheEvent - ) throws Exception + new PathChildrenCacheListener() { - if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - if (assignSegmentLatch.getCount() > 0) { - //Coordinator should try to assign segment to druidServer historical - //Simulate historical loading segment - druidServer.addDataSegment(dataSegment.getIdentifier(), dataSegment); - assignSegmentLatch.countDown(); - } else { - Assert.fail("The same segment is assigned to the same server multiple times"); + @Override + public void childEvent( + CuratorFramework curatorFramework, PathChildrenCacheEvent pathChildrenCacheEvent + ) throws Exception + { + if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { + if (assignSegmentLatch.getCount() > 0) { + //Coordinator should try to assign segment to druidServer historical + //Simulate historical loading segment + druidServer.addDataSegment(dataSegment.getIdentifier(), dataSegment); + assignSegmentLatch.countDown(); + } else { + Assert.fail("The same segment is assigned to the same server multiple times"); + } } } } - } ); pathChildrenCache.start(); diff --git a/server/src/test/java/io/druid/server/coordinator/rules/IntervalLoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/IntervalLoadRuleTest.java index a13209316a4..59ae0d4bbe2 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/IntervalLoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/IntervalLoadRuleTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - + package io.druid.server.coordinator.rules; import com.fasterxml.jackson.databind.ObjectMapper; @@ -26,56 +26,59 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import org.junit.Assert; import org.junit.Test; - - /** - */ - public class IntervalLoadRuleTest - { - @Test - public void testSerde() throws Exception - { - IntervalLoadRule rule = new IntervalLoadRule( - Intervals.of("0/3000"), - ImmutableMap.of(DruidServer.DEFAULT_TIER, 2) - ); - - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); - - Assert.assertEquals(rule, reread); - } - @Test - public void testSerdeNullTieredReplicants() throws Exception - { - IntervalLoadRule rule = new IntervalLoadRule( - Intervals.of("0/3000"), null - ); +/** + */ +public class IntervalLoadRuleTest +{ + @Test + public void testSerde() throws Exception + { + IntervalLoadRule rule = new IntervalLoadRule( + Intervals.of("0/3000"), + ImmutableMap.of(DruidServer.DEFAULT_TIER, 2) + ); - ObjectMapper jsonMapper = new DefaultObjectMapper(); - Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + ObjectMapper jsonMapper = new DefaultObjectMapper(); + Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); - Assert.assertEquals(rule, reread); - Assert.assertEquals(ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), rule.getTieredReplicants()); - } + Assert.assertEquals(rule, reread); + } - @Test - public void testMappingNullTieredReplicants() throws Exception - { - String inputJson = " {\n" - + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" - + " \"type\": \"loadByInterval\"\n" - + " }"; - String expectedJson = "{\n" - + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" - + " \"tieredReplicants\": {\n" - + " \"" + DruidServer.DEFAULT_TIER + "\": " + DruidServer.DEFAULT_NUM_REPLICANTS + "\n" - + " },\n" - + " \"type\": \"loadByInterval\"\n" - + " }"; - ObjectMapper jsonMapper = new DefaultObjectMapper(); - IntervalLoadRule inputIntervalLoadRule = jsonMapper.readValue(inputJson, IntervalLoadRule.class); - IntervalLoadRule expectedIntervalLoadRule = jsonMapper.readValue(expectedJson, IntervalLoadRule.class); - Assert.assertEquals(expectedIntervalLoadRule, inputIntervalLoadRule); - } - } + @Test + public void testSerdeNullTieredReplicants() throws Exception + { + IntervalLoadRule rule = new IntervalLoadRule( + Intervals.of("0/3000"), null + ); + + ObjectMapper jsonMapper = new DefaultObjectMapper(); + Rule reread = jsonMapper.readValue(jsonMapper.writeValueAsString(rule), Rule.class); + + Assert.assertEquals(rule, reread); + Assert.assertEquals( + ImmutableMap.of(DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_NUM_REPLICANTS), + rule.getTieredReplicants() + ); + } + + @Test + public void testMappingNullTieredReplicants() throws Exception + { + String inputJson = " {\n" + + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" + + " \"type\": \"loadByInterval\"\n" + + " }"; + String expectedJson = "{\n" + + " \"interval\": \"0000-01-01T00:00:00.000-05:50:36/3000-01-01T00:00:00.000-06:00\",\n" + + " \"tieredReplicants\": {\n" + + " \"" + DruidServer.DEFAULT_TIER + "\": " + DruidServer.DEFAULT_NUM_REPLICANTS + "\n" + + " },\n" + + " \"type\": \"loadByInterval\"\n" + + " }"; + ObjectMapper jsonMapper = new DefaultObjectMapper(); + IntervalLoadRule inputIntervalLoadRule = jsonMapper.readValue(inputJson, IntervalLoadRule.class); + IntervalLoadRule expectedIntervalLoadRule = jsonMapper.readValue(expectedJson, IntervalLoadRule.class); + Assert.assertEquals(expectedIntervalLoadRule, inputIntervalLoadRule); + } +} diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 1e9206f0b23..05f4fc2ce1c 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -60,6 +60,7 @@ import java.net.HttpURLConnection; import java.net.URL; import java.nio.charset.Charset; import java.util.EnumSet; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; @@ -173,14 +174,13 @@ public class JettyTest extends BaseJettyTest e.printStackTrace(); } finally { - System.out - .println( - "Response time client" - + (System.currentTimeMillis() - startTime) - + "time taken for getting future" - + (System.currentTimeMillis() - startTime2) - + "Counter " + count.incrementAndGet() - ); + System.out.printf( + Locale.ENGLISH, + "Response time client%dtime taken for getting future%dCounter %d%n", + System.currentTimeMillis() - startTime, + System.currentTimeMillis() - startTime2, + count.incrementAndGet() + ); latch.countDown(); } diff --git a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java index 5efba297c96..74e99c97e51 100644 --- a/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/SingleDimensionShardSpecTest.java @@ -19,13 +19,11 @@ package io.druid.server.shard; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; - import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.Pair; @@ -102,16 +100,9 @@ public class SingleDimensionShardSpecTest SingleDimensionShardSpec spec = entry.getKey(); for (Pair> pair : entry.getValue()) { final InputRow inputRow = new MapBasedInputRow( - 0, ImmutableList.of("billy"), Maps.transformValues( - pair.rhs, new Function() - { - @Override - public Object apply(String input) - { - return input; - } - } - ) + 0, + ImmutableList.of("billy"), + Maps.transformValues(pair.rhs, input -> input) ); Assert.assertEquals(StringUtils.format("spec[%s], row[%s]", spec, inputRow), pair.lhs, spec.isInChunk(inputRow.getTimestampFromEpoch(), inputRow)); } diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 6d21bf3d1e8..c00b288ddf0 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -28,7 +28,9 @@ import io.druid.guice.ExtensionsConfig; import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; +import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.ServiceLoader; /** @@ -51,32 +53,38 @@ public class Main .withDefaultCommand(Help.class) .withCommands(Help.class, Version.class); + List> serverCommands = Arrays.asList( + CliCoordinator.class, + CliHistorical.class, + CliBroker.class, + CliRealtime.class, + CliOverlord.class, + CliMiddleManager.class, + CliRouter.class + ); builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands( - CliCoordinator.class, CliHistorical.class, CliBroker.class, - CliRealtime.class, CliOverlord.class, CliMiddleManager.class, - CliRouter.class - ); + .withCommands(serverCommands); builder.withGroup("example") .withDescription("Run an example") .withDefaultCommand(Help.class) .withCommands(CliRealtimeExample.class); + List> toolCommands = Arrays.asList( + DruidJsonValidator.class, + PullDependencies.class, + CreateTables.class, + InsertSegment.class, + DumpSegment.class, + ResetCluster.class, + ValidateSegments.class + ); builder.withGroup("tools") .withDescription("Various tools for working with Druid") .withDefaultCommand(Help.class) - .withCommands( - DruidJsonValidator.class, - PullDependencies.class, - CreateTables.class, - InsertSegment.class, - DumpSegment.class, - ResetCluster.class, - ValidateSegments.class - ); + .withCommands(toolCommands); builder.withGroup("index") .withDescription("Run indexing for druid") diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index b83a93bdbe9..4c55781634b 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -154,15 +154,15 @@ public class DruidJsonValidatorTest ), new RealtimeIOConfig( new LocalFirehoseFactory(new File("lol"), "rofl", null), new PlumberSchool() - { - @Override - public Plumber findPlumber( - DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics - ) - { - return null; - } - }, + { + @Override + public Plumber findPlumber( + DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics + ) + { + return null; + } + }, null ), diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java index e617e6b8b3a..621a4fa9aae 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java @@ -158,7 +158,7 @@ public class DruidStatement implements Closeable final String query, final long maxRowCount, final AuthenticationResult authenticationResult - ) + ) { try (final DruidPlanner planner = plannerFactory.createPlanner(queryContext)) { synchronized (lock) { diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index a0246920ae4..fb273c066cb 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -197,11 +197,12 @@ public class DruidSchema extends AbstractSchema // Fuzz a bit to spread load out when we have multiple brokers. final long nextRefresh = nextRefreshNoFuzz + (long) ((nextRefreshNoFuzz - lastRefresh) * 0.10); - while (!( - isServerViewInitialized - && (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) - && (refreshImmediately || nextRefresh < System.currentTimeMillis()) - )) { + while (true) { + if (isServerViewInitialized && + (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && + (refreshImmediately || nextRefresh < System.currentTimeMillis())) { + break; + } lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis())); } diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index 75e75f33519..5f1840b81dd 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -163,7 +163,7 @@ public class DruidAvaticaHandlerTest ); final DruidAvaticaHandler handler = new DruidAvaticaHandler( druidMeta, - new DruidNode("dummy", "dummy", 1, null, new ServerConfig()), + new DruidNode("dummy", "dummy", 1, null, new ServerConfig()), new AvaticaMonitor() ); final int port = new Random().nextInt(9999) + 10000; diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 9839fb93e80..67023ad13ec 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -390,8 +390,8 @@ public class CalciteQueryTest ImmutableList.of( new Object[]{ "BindableProject(COLUMN_NAME=[$3], DATA_TYPE=[$7])\n" - + " BindableFilter(condition=[AND(=($1, 'druid'), =($2, 'foo'))])\n" - + " BindableTableScan(table=[[INFORMATION_SCHEMA, COLUMNS]])\n" + + " BindableFilter(condition=[AND(=($1, 'druid'), =($2, 'foo'))])\n" + + " BindableTableScan(table=[[INFORMATION_SCHEMA, COLUMNS]])\n" } ) ); @@ -704,9 +704,9 @@ public class CalciteQueryTest ImmutableList.of( new Object[]{ "BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n" - + " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); @@ -1569,16 +1569,16 @@ public class CalciteQueryTest ImmutableList.of( new Object[]{ "DruidQueryRel(query=[{" - + "\"queryType\":\"timeseries\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"descending\":false," - + "\"virtualColumns\":[]," - + "\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\",\"extractionFn\":null},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]}," - + "\"granularity\":{\"type\":\"all\"}," - + "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}]," - + "\"postAggregations\":[]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n" + + "\"queryType\":\"timeseries\"," + + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"descending\":false," + + "\"virtualColumns\":[]," + + "\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\",\"extractionFn\":null},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]}," + + "\"granularity\":{\"type\":\"all\"}," + + "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}]," + + "\"postAggregations\":[]," + + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n" } ) ); @@ -3229,8 +3229,8 @@ public class CalciteQueryTest ImmutableList.of( new Object[]{ "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null},{\"type\":\"count\",\"name\":\"a1\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" - + " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"d1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"d1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); @@ -3460,8 +3460,8 @@ public class CalciteQueryTest ImmutableList.of( new Object[]{ "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" - + " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], leftExpressions=[[DruidExpression{simpleExtraction=null, expression='substring(\"dim2\", 0, 1)'}]], rightKeys=[[0]])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], leftExpressions=[[DruidExpression{simpleExtraction=null, expression='substring(\"dim2\", 0, 1)'}]], rightKeys=[[0]])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index e9d12f65cca..a3a328f3c68 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -138,15 +138,13 @@ public class CalciteTests // This Module is just to get a LookupReferencesManager with a usable "lookyloo" lookup. - binder.bind(LookupReferencesManager.class) - .toInstance( - TestExprMacroTable.createTestLookupReferencesManager( - ImmutableMap.of( - "a", "xa", - "abc", "xabc" - ) - ) - ); + LookupReferencesManager testLookupReferencesManager = TestExprMacroTable.createTestLookupReferencesManager( + ImmutableMap.of( + "a", "xa", + "abc", "xabc" + ) + ); + binder.bind(LookupReferencesManager.class).toInstance(testLookupReferencesManager); } } );