From 80f5e89a116a93714417d7944ab4cd9e001bdec5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 Dec 2017 15:10:19 +0900 Subject: [PATCH 1/7] Fix DoublesSketchComplexMetricSerde.getSerializer() (#5140) --- .../quantiles/DoublesSketchComplexMetricSerde.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java index fa35700c6030..ab54f7d0b0b6 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/quantiles/DoublesSketchComplexMetricSerde.java @@ -27,12 +27,12 @@ import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.nio.ByteBuffer; @@ -109,9 +109,8 @@ public void deserializeColumn(final ByteBuffer buffer, final ColumnBuilder build // support large columns @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } - } From 4365390310c1b397bb3a24ea9c94b57559fa19e7 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 6 Dec 2017 15:54:48 -0800 Subject: [PATCH 2/7] Remove duplicate fastutil dependency in processing pom.xml (#5142) --- processing/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processing/pom.xml b/processing/pom.xml index f1253cfdbcef..fc084247c28c 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -98,10 +98,6 @@ org.ow2.asm asm-commons - - it.unimi.dsi - fastutil - From 4f5e2b45496559815078d5e61b4dab718a56672f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 6 Dec 2017 18:37:02 -0800 Subject: [PATCH 3/7] Fix some unemitted alerts. (#5141) --- .../kafka/KafkaEightSimpleConsumerFirehoseFactory.java | 2 +- .../java/io/druid/indexing/overlord/RemoteTaskRunner.java | 2 +- .../java/io/druid/metadata/SQLMetadataSegmentManager.java | 2 +- .../main/java/io/druid/segment/realtime/RealtimeManager.java | 3 ++- .../io/druid/server/coordination/SegmentLoadDropHandler.java | 5 +++-- 5 files changed, 8 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java index a28c47039eb1..0c53007311d8 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java +++ b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/io/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java @@ -141,7 +141,7 @@ private Map loadOffsetFromPreviousMetaData(Object lastCommit) } log.info("Loaded offset map[%s]", offsetMap); } else { - log.makeAlert("Unable to cast lastCommit to Map for feed [%s]", feed); + log.makeAlert("Unable to cast lastCommit to Map for feed [%s]", feed).emit(); } return offsetMap; } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index c2f2a1d78330..26cdfb718130 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -880,7 +880,7 @@ private boolean announceTask( task.getId(), elapsed, config.getTaskAssignmentTimeout() - ); + ).emit(); taskComplete(taskRunnerWorkItem, theZkWorker, TaskStatus.failure(task.getId())); break; } diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java index 1a04d42c632d..eca6620f17a4 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java @@ -458,7 +458,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) )); } catch (IOException e) { - log.makeAlert(e, "Failed to read segment from db."); + log.makeAlert(e, "Failed to read segment from db.").emit(); return null; } } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index d9428f483fc9..73bf34fb0b28 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -362,7 +362,8 @@ private boolean runFirehoseV2(FirehoseV2 firehose) throws Exception } catch (Exception e) { log.makeAlert(e, "Unknown exception, Ignoring and continuing.") - .addData("inputRow", inputRow); + .addData("inputRow", inputRow) + .emit(); } try { diff --git a/server/src/main/java/io/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/io/druid/server/coordination/SegmentLoadDropHandler.java index a95fbb7df8e6..1e84da5e20a9 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentLoadDropHandler.java @@ -376,12 +376,13 @@ public void run() if (failedSegments.size() > 0) { log.makeAlert("%,d errors seen while loading segments", failedSegments.size()) - .addData("failedSegments", failedSegments); + .addData("failedSegments", failedSegments) + .emit(); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - log.makeAlert(e, "LoadingInterrupted"); + log.makeAlert(e, "LoadingInterrupted").emit(); } backgroundSegmentAnnouncer.finishAnnouncing(); From 294bd3ffb3774c84bd82345791d4b36e8a220e3e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 11 Dec 2017 20:27:37 -0800 Subject: [PATCH 4/7] DruidCoordinatorBalancer: Fix log message. (#5152) --- .../server/coordinator/helper/DruidCoordinatorBalancer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 0ec2aeb494b2..88cd9b50a128 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -99,7 +99,7 @@ private void balanceTier( if (!currentlyMovingSegments.get(tier).isEmpty()) { reduceLifetimes(tier); - log.info("[%s]: Still waiting on %,d segments to be moved", tier, currentlyMovingSegments.size()); + log.info("[%s]: Still waiting on %,d segments to be moved", tier, currentlyMovingSegments.get(tier).size()); return; } From 64848c7ebfd5b95c528c0890ef47990de4e0a8f3 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 12 Dec 2017 22:41:40 +0300 Subject: [PATCH 5/7] DataSegment memory optimizations (#5094) * Deduplicate DataSegments contents (loadSpec's keys, dimensions and metrics lists as a whole) more aggressively; use ArrayMap instead of default LinkedHashMap for DataSegment.loadSpec, because they have only 3 entries on average; prune DataSegment.loadSpec on brokers * Fix DataSegmentTest * Refinements * Try to fix * Fix the second DataSegmentTest * Nullability * Fix tests * Fix tests, unify to use TestHelper.getJsonMapper() * Revert TestUtil as ServerTestHelper, fix tests * Add newline * Fix indexing tests * Fix s3 tests * Try to fix tests, remove lazy caching of ObjectMapper in TestHelper, rename TestHelper.getJsonMapper() to makeJsonMapper() * Fix HDFS tests * Fix HdfsDataSegmentPusherTest * Capitalize constant names --- api/pom.xml | 4 + .../java/io/druid/timeline/DataSegment.java | 139 +++++++++++++----- .../io/druid/timeline/DataSegmentUtils.java | 3 - .../java/io/druid/timeline/PruneLoadSpec.java | 40 +++++ .../io/druid/timeline/DataSegmentTest.java | 12 +- extensions-core/hdfs-storage/pom.xml | 7 + .../loading/HdfsDataSegmentFinderTest.java | 18 +-- .../hdfs/HdfsDataSegmentPusherTest.java | 5 +- .../indexing/kafka/KafkaIndexTaskTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../lookup/LoadingLookupFactoryTest.java | 2 +- extensions-core/s3-extensions/pom.xml | 7 + .../storage/s3/S3DataSegmentFinderTest.java | 20 +-- .../indexer/BatchDeltaIngestionTest.java | 6 +- .../indexer/HadoopDruidIndexerMapperTest.java | 2 +- ...cUpdateDatasourcePathSpecSegmentsTest.java | 16 +- .../hadoop/DatasourceIngestionSpecTest.java | 2 +- .../hadoop/DatasourceRecordReaderTest.java | 7 +- .../hadoop/WindowedDataSegmentTest.java | 4 +- .../indexer/path/DatasourcePathSpecTest.java | 3 +- .../io/druid/indexing/common/TestUtils.java | 5 +- .../actions/SegmentAllocateActionTest.java | 11 +- .../SegmentListUsedActionSerdeTest.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 2 +- .../IngestSegmentFirehoseFactoryTest.java | 2 +- .../helpers/TaskLogAutoCleanerConfigTest.java | 6 +- ...nWithAffinityWorkerSelectStrategyTest.java | 2 +- .../io/druid/guice/ExtensionsConfigTest.java | 6 +- .../java/io/druid/query/DataSourceTest.java | 2 +- .../druid/query/DefaultQueryMetricsTest.java | 4 +- .../query/QueryInterruptedExceptionTest.java | 2 +- .../io/druid/query/RetryQueryRunnerTest.java | 2 +- .../aggregation/AggregationTestHelper.java | 12 +- .../aggregation/DoubleMaxAggregationTest.java | 2 +- .../aggregation/DoubleMinAggregationTest.java | 2 +- .../aggregation/HistogramAggregatorTest.java | 2 +- .../query/aggregation/HistogramTest.java | 2 +- .../aggregation/LongMaxAggregationTest.java | 2 +- .../aggregation/LongMinAggregationTest.java | 2 +- .../HyperUniquesAggregatorFactoryTest.java | 2 +- .../ListFilteredDimensionSpecTest.java | 2 +- .../RegexFilteredDimensionSpecTest.java | 2 +- .../DefaultGroupByQueryMetricsTest.java | 4 +- .../query/groupby/GroupByQueryConfigTest.java | 2 +- .../druid/query/groupby/GroupByQueryTest.java | 2 +- .../groupby/orderby/DefaultLimitSpecTest.java | 2 +- .../druid/query/lookup/LookupConfigTest.java | 2 +- .../metadata/metadata/ColumnAnalysisTest.java | 2 +- .../query/search/SearchQueryRunnerTest.java | 4 +- .../DefaultTimeseriesQueryMetricsTest.java | 4 +- .../TimeseriesQueryQueryToolChestTest.java | 2 +- .../query/timeseries/TimeseriesQueryTest.java | 2 +- .../topn/AlphaNumericTopNMetricSpecTest.java | 2 +- .../topn/DefaultTopNQueryMetricsTest.java | 4 +- .../topn/DimensionTopNMetricSpecTest.java | 8 +- .../topn/TopNQueryQueryToolChestTest.java | 2 +- .../io/druid/query/topn/TopNQueryTest.java | 2 +- .../java/io/druid/segment/MetadataTest.java | 2 +- .../java/io/druid/segment/TestHelper.java | 6 +- .../DictionaryEncodedColumnPartSerdeTest.java | 2 +- .../segment/virtual/VirtualColumnsTest.java | 2 +- .../SegmentLoaderLocalCacheManager.java | 3 +- .../BrokerSegmentWatcherConfigTest.java | 2 +- .../io/druid/client/BrokerServerViewTest.java | 3 +- .../client/CoordinatorServerViewTest.java | 4 +- .../HttpServerInventoryViewConfigTest.java | 4 +- .../client/HttpServerInventoryViewTest.java | 4 +- .../client/BatchServerInventoryViewTest.java | 4 +- .../client/ImmutableSegmentLoadInfoTest.java | 4 +- .../druid/discovery/DataNodeServiceTest.java | 2 +- .../discovery/LookupNodeServiceTest.java | 2 +- .../discovery/WorkerNodeServiceTest.java | 2 +- ...exerSQLMetadataStorageCoordinatorTest.java | 4 +- .../metadata/MetadataSegmentManagerTest.java | 4 +- .../query/lookup/LookupSnapshotTakerTest.java | 2 +- .../segment/indexing/DataSchemaTest.java | 2 +- .../indexing/RealtimeTuningConfigTest.java | 4 +- .../segment/indexing/TransformSpecTest.java | 2 +- .../loading/LocalDataSegmentFinderTest.java | 111 ++++++-------- .../loading/LocalDataSegmentPusherTest.java | 4 +- .../segment/realtime/FireDepartmentTest.java | 2 +- .../plumber/CustomVersioningPolicyTest.java | 6 +- .../plumber/RealtimePlumberSchoolTest.java | 2 +- .../ServerTestHelper.java} | 27 +--- .../SegmentLoadDropHandlerTest.java | 4 +- .../coordination/ZkCoordinatorTest.java | 7 +- .../BatchDataSegmentAnnouncerTest.java | 4 +- .../coordinator/HttpLoadQueuePeonTest.java | 15 +- .../server/coordinator/LoadQueuePeonTest.java | 6 +- .../http/CoordinatorDynamicConfigTest.java | 2 +- .../server/shard/NumberedShardSpecTest.java | 8 +- .../{client => timeline}/DataSegmentTest.java | 13 +- .../HashBasedNumberedShardSpecTest.java | 36 +++-- .../src/main/java/io/druid/cli/CliBroker.java | 2 + .../druid/sql/calcite/http/SqlQueryTest.java | 2 +- .../druid/sql/calcite/util/CalciteTests.java | 6 +- 96 files changed, 417 insertions(+), 327 deletions(-) create mode 100644 api/src/main/java/io/druid/timeline/PruneLoadSpec.java rename server/src/test/java/io/druid/{TestUtil.java => server/ServerTestHelper.java} (67%) rename server/src/test/java/io/druid/{client => timeline}/DataSegmentTest.java (95%) diff --git a/api/pom.xml b/api/pom.xml index 04fdda41eb34..c41d8e100fbe 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -91,6 +91,10 @@ net.thisptr jackson-jq + + it.unimi.dsi + fastutil + junit diff --git a/api/src/main/java/io/druid/timeline/DataSegment.java b/api/src/main/java/io/druid/timeline/DataSegment.java index 0b4b2f9fe5c8..bfb2653378f5 100644 --- a/api/src/main/java/io/druid/timeline/DataSegment.java +++ b/api/src/main/java/io/druid/timeline/DataSegment.java @@ -19,18 +19,19 @@ package io.druid.timeline; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import com.google.common.base.Function; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Interner; import com.google.common.collect.Interners; -import com.google.common.collect.Iterables; +import com.google.inject.Inject; import io.druid.guice.annotations.PublicApi; import io.druid.jackson.CommaListJoinDeserializer; import io.druid.jackson.CommaListJoinSerializer; @@ -38,12 +39,15 @@ import io.druid.query.SegmentDescriptor; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; +import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -52,15 +56,13 @@ public class DataSegment implements Comparable { public static String delimiter = "_"; private final Integer binaryVersion; - private static final Interner interner = Interners.newWeakInterner(); - private static final Function internFun = new Function() - { - @Override - public String apply(String input) - { - return interner.intern(input); - } - }; + private static final Interner STRING_INTERNER = Interners.newWeakInterner(); + private static final Interner> DIMENSIONS_INTERNER = Interners.newWeakInterner(); + private static final Interner> METRICS_INTERNER = Interners.newWeakInterner(); + private static final Map PRUNED_LOAD_SPEC = ImmutableMap.of( + "load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space", + "" + ); public static String makeDataSegmentIdentifier( String dataSource, @@ -84,9 +86,23 @@ public static String makeDataSegmentIdentifier( return sb.toString(); } + /** + * This class is needed for optional injection of pruneLoadSpec, see + * github.com/google/guice/wiki/FrequentlyAskedQuestions#how-can-i-inject-optional-parameters-into-a-constructor + */ + @VisibleForTesting + public static class PruneLoadSpecHolder + { + @VisibleForTesting + public static final PruneLoadSpecHolder DEFAULT = new PruneLoadSpecHolder(); + + @Inject(optional = true) @PruneLoadSpec boolean pruneLoadSpec = false; + } + private final String dataSource; private final Interval interval; private final String version; + @Nullable private final Map loadSpec; private final List dimensions; private final List metrics; @@ -94,40 +110,63 @@ public static String makeDataSegmentIdentifier( private final long size; private final String identifier; + public DataSegment( + String dataSource, + Interval interval, + String version, + Map loadSpec, + List dimensions, + List metrics, + ShardSpec shardSpec, + Integer binaryVersion, + long size + ) + { + this( + dataSource, + interval, + version, + loadSpec, + dimensions, + metrics, + shardSpec, + binaryVersion, + size, + PruneLoadSpecHolder.DEFAULT + ); + } + @JsonCreator public DataSegment( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution - @JsonProperty("loadSpec") Map loadSpec, - @JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List dimensions, - @JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List metrics, - @JsonProperty("shardSpec") ShardSpec shardSpec, + @JsonProperty("loadSpec") @Nullable Map loadSpec, + @JsonProperty("dimensions") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List dimensions, + @JsonProperty("metrics") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List metrics, + @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, @JsonProperty("binaryVersion") Integer binaryVersion, - @JsonProperty("size") long size + @JsonProperty("size") long size, + @JacksonInject PruneLoadSpecHolder pruneLoadSpecHolder ) { - final Predicate nonEmpty = new Predicate() - { - @Override - public boolean apply(String input) - { - return input != null && !input.isEmpty(); - } - }; - - // dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing large numbers of segments. - this.dataSource = interner.intern(dataSource); + // dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing + // large numbers of segments. + this.dataSource = STRING_INTERNER.intern(dataSource); this.interval = interval; - this.loadSpec = loadSpec; + this.loadSpec = pruneLoadSpecHolder.pruneLoadSpec ? PRUNED_LOAD_SPEC : prepareLoadSpec(loadSpec); this.version = version; - this.dimensions = dimensions == null - ? ImmutableList.of() - : ImmutableList.copyOf(Iterables.transform(Iterables.filter(dimensions, nonEmpty), internFun)); - this.metrics = metrics == null - ? ImmutableList.of() - : ImmutableList.copyOf(Iterables.transform(Iterables.filter(metrics, nonEmpty), internFun)); + // Deduplicating dimensions and metrics lists as a whole because they are very likely the same for the same + // dataSource + this.dimensions = prepareDimensionsOrMetrics(dimensions, DIMENSIONS_INTERNER); + this.metrics = prepareDimensionsOrMetrics(metrics, METRICS_INTERNER); this.shardSpec = (shardSpec == null) ? NoneShardSpec.instance() : shardSpec; this.binaryVersion = binaryVersion; this.size = size; @@ -141,6 +180,35 @@ public boolean apply(String input) ); } + @Nullable + private Map prepareLoadSpec(@Nullable Map loadSpec) + { + if (loadSpec == null) { + return null; + } + // Load spec is just of 3 entries on average; HashMap/LinkedHashMap consumes much more memory than ArrayMap + Map result = new Object2ObjectArrayMap<>(loadSpec.size()); + for (Map.Entry e : loadSpec.entrySet()) { + result.put(STRING_INTERNER.intern(e.getKey()), e.getValue()); + } + return result; + } + + private List prepareDimensionsOrMetrics(@Nullable List list, Interner> interner) + { + if (list == null) { + return ImmutableList.of(); + } else { + List result = list + .stream() + .filter(s -> !Strings.isNullOrEmpty(s)) + .map(STRING_INTERNER::intern) + // TODO replace with ImmutableList.toImmutableList() when updated to Guava 21+ + .collect(Collectors.collectingAndThen(Collectors.toList(), ImmutableList::copyOf)); + return interner.intern(result); + } + } + /** * Get dataSource * @@ -158,6 +226,7 @@ public Interval getInterval() return interval; } + @Nullable @JsonProperty public Map getLoadSpec() { diff --git a/api/src/main/java/io/druid/timeline/DataSegmentUtils.java b/api/src/main/java/io/druid/timeline/DataSegmentUtils.java index 2457da3e1d80..a89e9709c838 100644 --- a/api/src/main/java/io/druid/timeline/DataSegmentUtils.java +++ b/api/src/main/java/io/druid/timeline/DataSegmentUtils.java @@ -30,9 +30,6 @@ import java.util.Objects; -/** - * identifier to DataSegment. - */ @PublicApi public class DataSegmentUtils { diff --git a/api/src/main/java/io/druid/timeline/PruneLoadSpec.java b/api/src/main/java/io/druid/timeline/PruneLoadSpec.java new file mode 100644 index 000000000000..758b5b5462b3 --- /dev/null +++ b/api/src/main/java/io/druid/timeline/PruneLoadSpec.java @@ -0,0 +1,40 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.timeline; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * This annnotation is used to inject a boolean parameter into a {@link DataSegment} constructor, which prescribes to + * drop deserialized "loadSpec" and don't store it in a field of a {@link DataSegment}. It's very useful on Brokers, + * because they store a lot of DataSegments in their heap, and loadSpec takes a lot of space, while it's not used on + * Brokers. + */ +@Target({ElementType.PARAMETER, ElementType.FIELD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface PruneLoadSpec +{ +} diff --git a/api/src/test/java/io/druid/timeline/DataSegmentTest.java b/api/src/test/java/io/druid/timeline/DataSegmentTest.java index b5c60d717bd8..4222775521f9 100644 --- a/api/src/test/java/io/druid/timeline/DataSegmentTest.java +++ b/api/src/test/java/io/druid/timeline/DataSegmentTest.java @@ -19,6 +19,7 @@ package io.druid.timeline; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -27,15 +28,16 @@ import com.google.common.collect.Sets; import io.druid.TestObjectMapper; import io.druid.data.input.InputRow; -import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.ShardSpec; import io.druid.timeline.partition.ShardSpecLookup; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -87,6 +89,14 @@ public Map> getDomain() }; } + @Before + public void setUp() + { + InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); + mapper.setInjectableValues(injectableValues); + } + @Test public void testV1Serialization() throws Exception { diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index 1a842ac76d4d..b787ad359692 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -173,6 +173,13 @@ ${project.parent.version} test + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + org.apache.hadoop hadoop-hdfs diff --git a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java index 70b5e0c59062..626124622527 100644 --- a/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java +++ b/extensions-core/hdfs-storage/src/test/java/io/druid/segment/loading/HdfsDataSegmentFinderTest.java @@ -23,9 +23,9 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.IOE; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import io.druid.storage.hdfs.HdfsDataSegmentFinder; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; @@ -53,7 +53,7 @@ public class HdfsDataSegmentFinderTest { - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static final ObjectMapper mapper = TestHelper.makeJsonMapper(); private static final String DESCRIPTOR_JSON = "descriptor.json"; private static final String INDEX_ZIP = "index.zip"; private static final DataSegment SEGMENT_1 = DataSegment @@ -278,20 +278,6 @@ public void testFindSegmentsFail() throws Exception hdfsDataSegmentFinder.findSegments(dataSourceDir.toString(), false); } - @Test(expected = SegmentLoadingException.class) - public void testFindSegmentsFail2() throws SegmentLoadingException - { - // will fail to desierialize descriptor.json because DefaultObjectMapper doesn't recognize NumberedShardSpec - final HdfsDataSegmentFinder hdfsDataSegmentFinder = new HdfsDataSegmentFinder(conf, new DefaultObjectMapper()); - try { - hdfsDataSegmentFinder.findSegments(dataSourceDir.toString(), false); - } - catch (SegmentLoadingException e) { - Assert.assertTrue(e.getCause() instanceof IOException); - throw e; - } - } - private String getDescriptorPath(DataSegment segment) { final Path indexzip = new Path(String.valueOf(segment.getLoadSpec().get("path"))); 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 cad4b9b2f1bc..95dd85bf2b9e 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 @@ -92,7 +92,10 @@ public void setUp() throws IOException } static { objectMapper = new TestObjectMapper(); - objectMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, objectMapper)); + InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(ObjectMapper.class, objectMapper); + injectableValues.addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); + objectMapper.setInjectableValues(injectableValues); } @Test 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 9d94127d4de3..69797ead923e 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 @@ -168,7 +168,7 @@ public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); - private static final ObjectMapper objectMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final long POLL_RETRY_MS = 100; private static TestingCluster zkServer; diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index cd159edf2731..84bbe54cb6d3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -112,7 +112,7 @@ @RunWith(Parameterized.class) public class KafkaSupervisorTest extends EasyMockSupport { - private static final ObjectMapper objectMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final String TOPIC_PREFIX = "testTopic"; private static final String DATASOURCE = "testDS"; private static final int NUM_PARTITIONS = 3; diff --git a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupFactoryTest.java b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupFactoryTest.java index a051e67ea00a..159fabe7bbd3 100644 --- a/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupFactoryTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/io/druid/server/lookup/LoadingLookupFactoryTest.java @@ -104,7 +104,7 @@ public void testGet() @Test public void testSerDeser() throws IOException { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); LoadingLookupFactory loadingLookupFactory = new LoadingLookupFactory( new MockDataFetcher(), new OnHeapLoadingCache( diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 2c89929620a6..a72f5b166ab7 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -75,6 +75,13 @@ ${project.parent.version} test + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + junit junit 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 67af162759be..f0df427ccb29 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 @@ -30,8 +30,8 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; @@ -52,7 +52,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.io.IOException; import java.io.InputStream; import java.util.List; import java.util.Map; @@ -60,7 +59,7 @@ public class S3DataSegmentFinderTest { - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static final ObjectMapper mapper = TestHelper.makeJsonMapper(); private static final DataSegment SEGMENT_1 = DataSegment .builder() @@ -247,21 +246,6 @@ public void testFindSegmentsFail() throws SegmentLoadingException, ServiceExcept s3DataSegmentFinder.findSegments("", false); } - @Test(expected = SegmentLoadingException.class) - public void testFindSegmentsFail2() throws SegmentLoadingException - { - final S3DataSegmentFinder s3DataSegmentFinder = new S3DataSegmentFinder( - mockS3Client, config, new DefaultObjectMapper()); - - try { - s3DataSegmentFinder.findSegments("", false); - } - catch (SegmentLoadingException e) { - Assert.assertTrue(e.getCause() instanceof IOException); - throw e; - } - } - @Test public void testFindSegmentsWithmaxListingLength() throws SegmentLoadingException { diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 5ba57ac90d91..f16dbd41f7e3 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -80,12 +80,14 @@ public class BatchDeltaIngestionTest static { MAPPER = new DefaultObjectMapper(); MAPPER.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); - InjectableValues inject = new InjectableValues.Std().addValue(ObjectMapper.class, MAPPER); + InjectableValues inject = new InjectableValues.Std() + .addValue(ObjectMapper.class, MAPPER) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); MAPPER.setInjectableValues(inject); INDEX_IO = HadoopDruidIndexerConfig.INDEX_IO; try { - SEGMENT = new DefaultObjectMapper() + SEGMENT = MAPPER .readValue( BatchDeltaIngestionTest.class.getClassLoader().getResource("test-segment/descriptor.json"), DataSegment.class diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java index 6aa5177c85d8..d7c7fdffd553 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -55,7 +55,7 @@ public class HadoopDruidIndexerMapperTest { - private static final ObjectMapper JSON_MAPPER = TestHelper.getJsonMapper(); + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static final DataSchema DATA_SCHEMA = new DataSchema( "test_ds", JSON_MAPPER.convertValue( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index e9d92e55c0ee..50f51752cbec 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -60,7 +60,9 @@ public HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest() { jsonMapper = new DefaultObjectMapper(); jsonMapper.setInjectableValues( - new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper) + new InjectableValues.Std() + .addValue(ObjectMapper.class, jsonMapper) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) ); } @@ -80,7 +82,7 @@ public HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest() ); @Test - public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithNoDatasourcePathSpec() throws Exception + public void testUpdateSegmentListIfDatasourcePathSpecIsUsedWithNoDatasourcePathSpec() throws Exception { PathSpec pathSpec = new StaticPathSpec("/xyz", null); HadoopDruidIndexerConfig config = testRunUpdateSegmentListIfDatasourcePathSpecIsUsed(pathSpec, null); @@ -88,7 +90,7 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithNoDatasourcePathS } @Test - public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePathSpec() throws Exception + public void testUpdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePathSpec() throws Exception { PathSpec pathSpec = new DatasourcePathSpec( jsonMapper, @@ -107,7 +109,7 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePat } @Test - public void testupdateSegmentListIfDatasourcePathSpecWithMatchingUserSegments() throws Exception + public void testUpdateSegmentListIfDatasourcePathSpecWithMatchingUserSegments() throws Exception { PathSpec pathSpec = new DatasourcePathSpec( jsonMapper, @@ -136,7 +138,7 @@ public void testupdateSegmentListIfDatasourcePathSpecWithMatchingUserSegments() } @Test(expected = IOException.class) - public void testupdateSegmentListThrowsExceptionWithUserSegmentsMismatch() throws Exception + public void testUpdateSegmentListThrowsExceptionWithUserSegmentsMismatch() throws Exception { PathSpec pathSpec = new DatasourcePathSpec( jsonMapper, @@ -161,7 +163,7 @@ public void testupdateSegmentListThrowsExceptionWithUserSegmentsMismatch() throw } @Test - public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePathSpecAndPartialInterval() + public void testUpdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePathSpecAndPartialInterval() throws Exception { PathSpec pathSpec = new DatasourcePathSpec( @@ -191,7 +193,7 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePat } @Test - public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithMultiplePathSpec() throws Exception + public void testUpdateSegmentListIfDatasourcePathSpecIsUsedWithMultiplePathSpec() throws Exception { PathSpec pathSpec = new MultiplePathSpec( ImmutableList.of( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java index cee6f6833222..d0dbd4ceb690 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java @@ -36,7 +36,7 @@ */ public class DatasourceIngestionSpecTest { - private static final ObjectMapper MAPPER = TestHelper.getJsonMapper(); + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @Test public void testSingleIntervalSerde() throws Exception diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java index 237ecfba03f5..65947dac3d0a 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.Lists; import io.druid.data.input.InputRow; import io.druid.indexer.HadoopDruidIndexerConfig; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.timeline.DataSegment; @@ -35,6 +34,7 @@ import org.junit.Assert; import org.junit.Test; +import java.net.URL; import java.util.List; import java.util.Map; @@ -45,8 +45,9 @@ public class DatasourceRecordReaderTest @Test public void testSanity() throws Exception { - DataSegment segment = new DefaultObjectMapper() - .readValue(this.getClass().getClassLoader().getResource("test-segment/descriptor.json"), DataSegment.class) + URL segmentDesciptor = this.getClass().getClassLoader().getResource("test-segment/descriptor.json"); + DataSegment segment = HadoopDruidIndexerConfig.JSON_MAPPER + .readValue(segmentDesciptor, DataSegment.class) .withLoadSpec( ImmutableMap.of( "type", diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/WindowedDataSegmentTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/WindowedDataSegmentTest.java index f350bb7ff615..4fd5b0fd1fee 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/WindowedDataSegmentTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/WindowedDataSegmentTest.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; @@ -34,7 +34,7 @@ public class WindowedDataSegmentTest { - private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); private static final DataSegment SEGMENT = new DataSegment( "test1", Intervals.of("2000/3000"), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java index ab1b0780a9ed..49429418efcf 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java @@ -48,6 +48,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.server.DruidNode; @@ -178,7 +179,7 @@ public void testAddInputPaths() throws Exception { HadoopDruidIndexerConfig hadoopIndexerConfig = makeHadoopDruidIndexerConfig(); - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); DatasourcePathSpec pathSpec = new DatasourcePathSpec( mapper, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 00c09435a9a2..ad8da3bfe121 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Stopwatch; -import io.druid.TestUtil; import io.druid.guice.ServerModule; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; @@ -38,6 +37,7 @@ import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; +import io.druid.timeline.DataSegment; import java.util.List; import java.util.concurrent.TimeUnit; @@ -46,7 +46,7 @@ */ public class TestUtils { - private static final Logger log = new Logger(TestUtil.class); + private static final Logger log = new Logger(TestUtils.class); private final ObjectMapper jsonMapper; private final IndexMergerV9 indexMergerV9; @@ -82,6 +82,7 @@ public int columnCacheSizeBytes() .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) .addValue(AuthConfig.class, new AuthConfig()) .addValue(AuthorizerMapper.class, null) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java index 1cfc154c46be..b7e82324739c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -138,16 +138,7 @@ public void testManySegmentsSameInterval() throws Exception final TaskLock partyLock = Iterables.getOnlyElement( FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) + .filter(input -> input.getInterval().contains(PARTY_TIME)) ); assertSameIdentifier( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java index ce83eabc0c76..32deac0ebabf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import io.druid.TestUtil; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -33,7 +33,7 @@ */ public class SegmentListUsedActionSerdeTest { - private static final ObjectMapper MAPPER = TestUtil.MAPPER; + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @Test public void testSingleIntervalSerde() throws Exception 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 bc0b0328ecba..6bea964f163e 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 @@ -893,7 +893,7 @@ private RealtimeIndexTask makeRealtimeTask( ObjectMapper objectMapper = new DefaultObjectMapper(); DataSchema dataSchema = new DataSchema( "test_ds", - TestHelper.getJsonMapper().convertValue( + TestHelper.makeJsonMapper().convertValue( new MapInputRowParser( new TimeAndDimsParseSpec( new TimestampSpec("t", "auto", 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 30a84e1ca85e..624ddc4cc425 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 @@ -132,7 +132,7 @@ public class IngestSegmentFirehoseFactoryTest static { TestUtils testUtils = new TestUtils(); - MAPPER = setupInjectablesInObjectMapper(TestHelper.getJsonMapper()); + MAPPER = setupInjectablesInObjectMapper(TestHelper.makeJsonMapper()); INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); INDEX_IO = testUtils.getTestIndexIO(); TASK_STORAGE = new HeapMemoryTaskStorage( diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/helpers/TaskLogAutoCleanerConfigTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/helpers/TaskLogAutoCleanerConfigTest.java index 23b4455e8888..75ae2b15213e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/helpers/TaskLogAutoCleanerConfigTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/helpers/TaskLogAutoCleanerConfigTest.java @@ -20,7 +20,7 @@ package io.druid.indexing.overlord.helpers; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.TestUtil; +import io.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; @@ -38,7 +38,7 @@ public void testSerde() throws Exception + " \"durationToRetain\": 30\n" + "}"; - ObjectMapper mapper = TestUtil.MAPPER; + ObjectMapper mapper = TestHelper.makeJsonMapper(); TaskLogAutoCleanerConfig config = mapper.readValue( mapper.writeValueAsString( @@ -60,7 +60,7 @@ public void testSerdeWithDefaults() throws Exception { String json = "{}"; - ObjectMapper mapper = TestUtil.MAPPER; + ObjectMapper mapper = TestHelper.makeJsonMapper(); TaskLogAutoCleanerConfig config = mapper.readValue( mapper.writeValueAsString( 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 dc74c225432a..d954971eb1e3 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 @@ -142,7 +142,7 @@ public void testIsolation() throws Exception @Test public void testSerde() throws Exception { - final ObjectMapper objectMapper = TestHelper.getJsonMapper(); + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); final EqualDistributionWorkerSelectStrategy strategy = new EqualDistributionWithAffinityWorkerSelectStrategy( new AffinityConfig(ImmutableMap.of("foo", ImmutableSet.of("localhost")), false) ); diff --git a/processing/src/test/java/io/druid/guice/ExtensionsConfigTest.java b/processing/src/test/java/io/druid/guice/ExtensionsConfigTest.java index 61f6c860b2da..daad3b8cba12 100644 --- a/processing/src/test/java/io/druid/guice/ExtensionsConfigTest.java +++ b/processing/src/test/java/io/druid/guice/ExtensionsConfigTest.java @@ -35,7 +35,7 @@ public class ExtensionsConfigTest public void testSerdeWithDefaults() throws Exception { String json = "{}"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); ExtensionsConfig config = mapper.readValue( mapper.writeValueAsString( @@ -61,7 +61,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"hadoopContainerDruidClasspath\": \"testHadoopContainerClasspath\",\n" + " \"loadList\": [\"b\",\"a\"]\n" + "}"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); ExtensionsConfig config = mapper.readValue( mapper.writeValueAsString( @@ -91,7 +91,7 @@ public void testLoadList() throws Exception + " \"hadoopContainerDruidClasspath\": \"testHadoopContainerClasspath\",\n" + " \"loadList\": [\"b\",\"b\",\"a\",\"c\",\"d\",\"a\"]\n" + "}"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); ExtensionsConfig config = mapper.readValue( mapper.writeValueAsString( diff --git a/processing/src/test/java/io/druid/query/DataSourceTest.java b/processing/src/test/java/io/druid/query/DataSourceTest.java index d99c5297cffb..d24d6ed79895 100644 --- a/processing/src/test/java/io/druid/query/DataSourceTest.java +++ b/processing/src/test/java/io/druid/query/DataSourceTest.java @@ -35,7 +35,7 @@ public class DataSourceTest { - private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @Test public void testSerialization() throws IOException diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index aee5c6e8fb4c..add3360d107c 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -50,7 +50,7 @@ public void testDefaultQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(TestHelper.getJsonMapper()); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(TestHelper.makeJsonMapper()); TopNQuery query = new TopNQueryBuilder() .dataSource("xx") .granularity(Granularities.ALL) @@ -91,7 +91,7 @@ public void testDefaultQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(TestHelper.getJsonMapper()); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(TestHelper.makeJsonMapper()); testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } diff --git a/processing/src/test/java/io/druid/query/QueryInterruptedExceptionTest.java b/processing/src/test/java/io/druid/query/QueryInterruptedExceptionTest.java index 44bc39347087..7369c47f610c 100644 --- a/processing/src/test/java/io/druid/query/QueryInterruptedExceptionTest.java +++ b/processing/src/test/java/io/druid/query/QueryInterruptedExceptionTest.java @@ -31,7 +31,7 @@ public class QueryInterruptedExceptionTest { - private static final ObjectMapper MAPPER = TestHelper.getJsonMapper(); + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @Test public void testErrorCode() diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index 5e7262ba7786..5ec73f868ce2 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -66,7 +66,7 @@ public boolean isReturnPartialResults() } } - private final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index 27e2c5958564..06575a13673c 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -138,7 +138,7 @@ public static final AggregationTestHelper createGroupByQueryAggregationTestHelpe TemporaryFolder tempFolder ) { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(mapper, config); IndexIO indexIO = new IndexIO( @@ -170,7 +170,7 @@ public static final AggregationTestHelper createSelectQueryAggregationTestHelper TemporaryFolder tempFolder ) { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); mapper.setInjectableValues( new InjectableValues.Std().addValue( SelectQueryConfig.class, @@ -181,14 +181,14 @@ public static final AggregationTestHelper createSelectQueryAggregationTestHelper Supplier configSupplier = Suppliers.ofInstance(new SelectQueryConfig(true)); SelectQueryQueryToolChest toolchest = new SelectQueryQueryToolChest( - TestHelper.getJsonMapper(), + TestHelper.makeJsonMapper(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), configSupplier ); SelectQueryRunnerFactory factory = new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( - TestHelper.getJsonMapper(), + TestHelper.makeJsonMapper(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), configSupplier ), @@ -226,7 +226,7 @@ public static final AggregationTestHelper createTimeseriesQueryAggregationTestHe TemporaryFolder tempFolder ) { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest( QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() @@ -267,7 +267,7 @@ public static final AggregationTestHelper createTopNQueryAggregationTestHelper( TemporaryFolder tempFolder ) { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest( new TopNQueryConfig(), diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java index 12cea46fcdda..7fa6cbbd3575 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -42,7 +42,7 @@ public class DoubleMaxAggregationTest public DoubleMaxAggregationTest() throws Exception { String aggSpecJson = "{\"type\": \"doubleMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; - doubleMaxAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, DoubleMaxAggregatorFactory.class); + doubleMaxAggFactory = TestHelper.makeJsonMapper().readValue(aggSpecJson, DoubleMaxAggregatorFactory.class); } @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java index 3d4735609d78..54d3b8711d16 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java @@ -42,7 +42,7 @@ public class DoubleMinAggregationTest public DoubleMinAggregationTest() throws Exception { String aggSpecJson = "{\"type\": \"doubleMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; - doubleMinAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, DoubleMinAggregatorFactory.class); + doubleMinAggFactory = TestHelper.makeJsonMapper().readValue(aggSpecJson, DoubleMinAggregatorFactory.class); } @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/HistogramAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/HistogramAggregatorTest.java index f0f62ff1953b..cd89b6d11b77 100644 --- a/processing/src/test/java/io/druid/query/aggregation/HistogramAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/HistogramAggregatorTest.java @@ -40,7 +40,7 @@ private void aggregate(TestFloatColumnSelector selector, Aggregator agg) @Test public void testSerde() throws Exception { - final ObjectMapper objectMapper = TestHelper.getJsonMapper(); + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); String json0 = "{\"type\": \"histogram\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; HistogramAggregatorFactory agg0 = objectMapper.readValue(json0, HistogramAggregatorFactory.class); Assert.assertEquals(ImmutableList.of(), agg0.getBreaks()); diff --git a/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java b/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java index 4d0a6d2d31ea..b8d934df72c4 100644 --- a/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java @@ -86,7 +86,7 @@ public void testAsVisual() throws Exception Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0}; Double[] visualCounts = {123., 4., 56., 7.}; - ObjectMapper objectMapper = TestHelper.getJsonMapper(); + ObjectMapper objectMapper = TestHelper.makeJsonMapper(); String json = objectMapper.writeValueAsString(h.asVisual()); Map expectedObj = Maps.newLinkedHashMap(); diff --git a/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java index e4e3a5490932..7419a2379cc7 100644 --- a/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java @@ -42,7 +42,7 @@ public class LongMaxAggregationTest public LongMaxAggregationTest() throws Exception { String aggSpecJson = "{\"type\": \"longMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; - longMaxAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, LongMaxAggregatorFactory.class); + longMaxAggFactory = TestHelper.makeJsonMapper().readValue(aggSpecJson, LongMaxAggregatorFactory.class); } @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java index 0b0227ec03f7..fdbdd7830bd3 100644 --- a/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java @@ -42,7 +42,7 @@ public class LongMinAggregationTest public LongMinAggregationTest() throws Exception { String aggSpecJson = "{\"type\": \"longMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; - longMinAggFactory = TestHelper.getJsonMapper().readValue(aggSpecJson, LongMinAggregatorFactory.class); + longMinAggFactory = TestHelper.makeJsonMapper().readValue(aggSpecJson, LongMinAggregatorFactory.class); } @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java index 7a54e87a5251..0239278fee9b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java @@ -183,7 +183,7 @@ public void testSerde() throws Exception true ); - final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); final AggregatorFactory factory2 = jsonMapper.readValue( jsonMapper.writeValueAsString(factory), AggregatorFactory.class diff --git a/processing/src/test/java/io/druid/query/dimension/ListFilteredDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/ListFilteredDimensionSpecTest.java index c61d4e48860b..df362244f6f6 100644 --- a/processing/src/test/java/io/druid/query/dimension/ListFilteredDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/ListFilteredDimensionSpecTest.java @@ -37,7 +37,7 @@ public class ListFilteredDimensionSpecTest @Test public void testSerde() throws Exception { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); //isWhitelist = true String jsonStr = "{\n" diff --git a/processing/src/test/java/io/druid/query/dimension/RegexFilteredDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/RegexFilteredDimensionSpecTest.java index 0dd8156b2cfc..6128be72d4a3 100644 --- a/processing/src/test/java/io/druid/query/dimension/RegexFilteredDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/RegexFilteredDimensionSpecTest.java @@ -36,7 +36,7 @@ public class RegexFilteredDimensionSpecTest @Test public void testSerde() throws Exception { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); String jsonStr = "{\n" + " \"type\": \"regexFiltered\",\n" diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java index 951c97d09d44..3d16b0108945 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -56,7 +56,7 @@ public void testDefaultGroupByQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.getJsonMapper()); + DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.makeJsonMapper()); GroupByQuery.Builder builder = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.dataSource) @@ -121,7 +121,7 @@ public void testDefaultGroupByQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.getJsonMapper()); + DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(TestHelper.makeJsonMapper()); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryConfigTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryConfigTest.java index eff05ca2033a..8134b577995d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryConfigTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryConfigTest.java @@ -29,7 +29,7 @@ public class GroupByQueryConfigTest { - private final ObjectMapper MAPPER = TestHelper.getJsonMapper(); + private final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); private final ImmutableMap CONFIG_MAP = ImmutableMap .builder() diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java index ff33fdead9d7..13a996f948d5 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryTest.java @@ -48,7 +48,7 @@ public class GroupByQueryTest { - private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @Test public void testQuerySerialization() throws IOException diff --git a/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java b/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java index 2337d3383792..c4bbebfe9c27 100644 --- a/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/orderby/DefaultLimitSpecTest.java @@ -68,7 +68,7 @@ public DefaultLimitSpecTest() @Test public void testSerde() throws Exception { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); //defaults String json = "{\"type\": \"default\"}"; diff --git a/processing/src/test/java/io/druid/query/lookup/LookupConfigTest.java b/processing/src/test/java/io/druid/query/lookup/LookupConfigTest.java index 67d6d27b0a94..6ea18de89756 100644 --- a/processing/src/test/java/io/druid/query/lookup/LookupConfigTest.java +++ b/processing/src/test/java/io/druid/query/lookup/LookupConfigTest.java @@ -31,7 +31,7 @@ public class LookupConfigTest { - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java b/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java index f2a86680f61d..59a87df6e5f6 100644 --- a/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java +++ b/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java @@ -26,7 +26,7 @@ public class ColumnAnalysisTest { - private final ObjectMapper MAPPER = TestHelper.getJsonMapper(); + private final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); private void assertSerDe(ColumnAnalysis analysis) throws Exception { diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 67b04ee338ed..d0d99751188f 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -111,8 +111,8 @@ public SearchQueryRunnerTest( public void testSearchHitSerDe() throws Exception { for (SearchHit hit : Arrays.asList(new SearchHit("dim1", "val1"), new SearchHit("dim2", "val2", 3))) { - SearchHit read = TestHelper.getJsonMapper().readValue( - TestHelper.getJsonMapper().writeValueAsString(hit), + SearchHit read = TestHelper.makeJsonMapper().readValue( + TestHelper.makeJsonMapper().writeValueAsString(hit), SearchHit.class ); Assert.assertEquals(hit, read); diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java index fa0288c95f19..912eaf6a55c9 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -46,7 +46,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.getJsonMapper()); + DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.makeJsonMapper()); TimeseriesQuery query = Druids .newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) @@ -89,7 +89,7 @@ public void testDefaultTimeseriesQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.getJsonMapper()); + DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(TestHelper.makeJsonMapper()); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index b23cb0fe28fe..84d8d952421a 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -92,7 +92,7 @@ public void testCacheStrategy() throws Exception Object preparedValue = strategy.prepareForCache().apply(result); - ObjectMapper objectMapper = TestHelper.getJsonMapper(); + ObjectMapper objectMapper = TestHelper.makeJsonMapper(); Object fromCacheValue = objectMapper.readValue( objectMapper.writeValueAsBytes(preparedValue), strategy.getCacheObjectClazz() diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java index 9ae8121b19d0..5c6dac5d855c 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryTest.java @@ -35,7 +35,7 @@ @RunWith(Parameterized.class) public class TimeseriesQueryTest { - private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @Parameterized.Parameters(name = "descending={0}") public static Iterable constructorFeeder() throws IOException diff --git a/processing/src/test/java/io/druid/query/topn/AlphaNumericTopNMetricSpecTest.java b/processing/src/test/java/io/druid/query/topn/AlphaNumericTopNMetricSpecTest.java index 4e66f8094b3b..c88338d4c1d5 100644 --- a/processing/src/test/java/io/druid/query/topn/AlphaNumericTopNMetricSpecTest.java +++ b/processing/src/test/java/io/druid/query/topn/AlphaNumericTopNMetricSpecTest.java @@ -103,7 +103,7 @@ public void testSerdeAlphaNumericTopNMetricSpec() throws IOException + " \"type\": \"alphaNumeric\",\n" + " \"previousStop\": \"test\"\n" + "}"; - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class); TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), AlphaNumericTopNMetricSpec.class); Assert.assertEquals(expectedMetricSpec, actualMetricSpec); diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java index e7df7d272e4d..a61b393a094c 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -52,7 +52,7 @@ public void testDefaultTopNQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.getJsonMapper()); + DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.makeJsonMapper()); TopNQuery query = new TopNQueryBuilder() .dataSource("xx") .granularity(Granularities.ALL) @@ -102,7 +102,7 @@ public void testDefaultTopNQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.getJsonMapper()); + DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(TestHelper.makeJsonMapper()); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } } diff --git a/processing/src/test/java/io/druid/query/topn/DimensionTopNMetricSpecTest.java b/processing/src/test/java/io/druid/query/topn/DimensionTopNMetricSpecTest.java index bc423b2464ae..292618ac821c 100644 --- a/processing/src/test/java/io/druid/query/topn/DimensionTopNMetricSpecTest.java +++ b/processing/src/test/java/io/druid/query/topn/DimensionTopNMetricSpecTest.java @@ -43,7 +43,7 @@ public void testSerdeAlphaNumericDimensionTopNMetricSpec() throws IOException + " \"ordering\": \"alphanumeric\",\n" + " \"previousStop\": \"test\"\n" + "}"; - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); Assert.assertEquals(expectedMetricSpec, actualMetricSpec); @@ -64,7 +64,7 @@ public void testSerdeLexicographicDimensionTopNMetricSpec() throws IOException + " \"ordering\": \"lexicographic\",\n" + " \"previousStop\": \"test\"\n" + "}"; - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); Assert.assertEquals(expectedMetricSpec, actualMetricSpec); @@ -85,7 +85,7 @@ public void testSerdeStrlenDimensionTopNMetricSpec() throws IOException + " \"ordering\": \"strlen\",\n" + " \"previousStop\": \"test\"\n" + "}"; - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); Assert.assertEquals(expectedMetricSpec, actualMetricSpec); @@ -106,7 +106,7 @@ public void testSerdeNumericDimensionTopNMetricSpec() throws IOException + " \"ordering\": \"numeric\",\n" + " \"previousStop\": \"test\"\n" + "}"; - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TopNMetricSpec actualMetricSpec = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); TopNMetricSpec actualMetricSpec1 = jsonMapper.readValue(jsonMapper.writeValueAsString(jsonMapper.readValue(jsonSpec1, TopNMetricSpec.class)), DimensionTopNMetricSpec.class); Assert.assertEquals(expectedMetricSpec, actualMetricSpec); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index e1d570bf1dd3..8938390986a5 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -95,7 +95,7 @@ public void testCacheStrategy() throws Exception result ); - ObjectMapper objectMapper = TestHelper.getJsonMapper(); + ObjectMapper objectMapper = TestHelper.makeJsonMapper(); Object fromCacheValue = objectMapper.readValue( objectMapper.writeValueAsBytes(preparedValue), strategy.getCacheObjectClazz() diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index f6af45ef1343..63312d4e58c6 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -51,7 +51,7 @@ public class TopNQueryTest { - private static final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private static final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @Test public void testQuerySerialization() throws IOException diff --git a/processing/src/test/java/io/druid/segment/MetadataTest.java b/processing/src/test/java/io/druid/segment/MetadataTest.java index 6ae55daadab4..3aba1dbc02d2 100644 --- a/processing/src/test/java/io/druid/segment/MetadataTest.java +++ b/processing/src/test/java/io/druid/segment/MetadataTest.java @@ -40,7 +40,7 @@ public class MetadataTest @Test public void testSerde() throws Exception { - ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); Metadata metadata = new Metadata(); metadata.put("k", "v"); diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index 7da66da3b590..6d27a4b2d9a5 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -35,6 +35,7 @@ import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.query.topn.TopNResultValue; import io.druid.segment.column.ColumnConfig; +import io.druid.timeline.DataSegment; import org.junit.Assert; import java.util.Iterator; @@ -47,7 +48,7 @@ */ public class TestHelper { - private static final ObjectMapper JSON_MAPPER = getJsonMapper(); + private static final ObjectMapper JSON_MAPPER = makeJsonMapper(); public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { @@ -70,13 +71,14 @@ public int columnCacheSizeBytes() ); } - public static ObjectMapper getJsonMapper() + public static ObjectMapper makeJsonMapper() { final ObjectMapper mapper = new DefaultObjectMapper(); mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) ); return mapper; } diff --git a/processing/src/test/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerdeTest.java b/processing/src/test/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerdeTest.java index a909b9d4ad1f..3735e7ff07df 100644 --- a/processing/src/test/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerdeTest.java @@ -41,7 +41,7 @@ public void testSerde() throws Exception + " \"byteOrder\": \"BIG_ENDIAN\"\n" + "}\n"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); DictionaryEncodedColumnPartSerde serde = (DictionaryEncodedColumnPartSerde) mapper.readValue( mapper.writeValueAsString( diff --git a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java index d6aa0fc517d2..943b50258ca6 100644 --- a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java @@ -252,7 +252,7 @@ public void testEqualsAndHashCode() throws Exception @Test public void testSerde() throws Exception { - final ObjectMapper mapper = TestHelper.getJsonMapper(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final ImmutableList theColumns = ImmutableList.of( new ExpressionVirtualColumn("expr", "x + y", ValueType.FLOAT, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("expr2", "x + z", ValueType.FLOAT, TestExprMacroTable.INSTANCE) diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java index 19a97f7816a2..3ef7547dd991 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderLocalCacheManager.java @@ -201,7 +201,8 @@ private void loadInLocationWithStartMarker(DataSegment segment, File storageDir) private void loadInLocation(DataSegment segment, File storageDir) throws SegmentLoadingException { - // LoadSpec isn't materialized until here so that any system can interpret Segment without having to have all the LoadSpec dependencies. + // LoadSpec isn't materialized until here so that any system can interpret Segment without having to have all the + // LoadSpec dependencies. final LoadSpec loadSpec = jsonMapper.convertValue(segment.getLoadSpec(), LoadSpec.class); final LoadSpec.LoadSpecResult result = loadSpec.loadSegment(storageDir); if (result.getSize() != segment.getSize()) { diff --git a/server/src/test/java/io/druid/client/BrokerSegmentWatcherConfigTest.java b/server/src/test/java/io/druid/client/BrokerSegmentWatcherConfigTest.java index 6a8da14726ca..5f0356e3d0bf 100644 --- a/server/src/test/java/io/druid/client/BrokerSegmentWatcherConfigTest.java +++ b/server/src/test/java/io/druid/client/BrokerSegmentWatcherConfigTest.java @@ -29,7 +29,7 @@ */ public class BrokerSegmentWatcherConfigTest { - private static final ObjectMapper MAPPER = TestHelper.getJsonMapper(); + private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @Test public void testSerde() throws Exception diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index 2dae5d4310a9..27a731fb2b78 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -39,6 +39,7 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.query.TableDataSource; +import io.druid.segment.TestHelper; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; @@ -75,7 +76,7 @@ public class BrokerServerViewTest extends CuratorTestBase public BrokerServerViewTest() { - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); zkPathsConfig = new ZkPathsConfig(); } diff --git a/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java index 8aa3fc9c5562..1a2adfd5c2c3 100644 --- a/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java +++ b/server/src/test/java/io/druid/client/CoordinatorServerViewTest.java @@ -27,10 +27,10 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.curator.CuratorTestBase; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.query.TableDataSource; +import io.druid.segment.TestHelper; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; @@ -66,7 +66,7 @@ public class CoordinatorServerViewTest extends CuratorTestBase public CoordinatorServerViewTest() { - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); zkPathsConfig = new ZkPathsConfig(); inventoryPath = zkPathsConfig.getLiveSegmentsPath(); } diff --git a/server/src/test/java/io/druid/client/HttpServerInventoryViewConfigTest.java b/server/src/test/java/io/druid/client/HttpServerInventoryViewConfigTest.java index 51b1c457ae23..4101887f4271 100644 --- a/server/src/test/java/io/druid/client/HttpServerInventoryViewConfigTest.java +++ b/server/src/test/java/io/druid/client/HttpServerInventoryViewConfigTest.java @@ -34,7 +34,7 @@ public void testDeserializationWithDefaults() throws Exception { String json = "{}"; - HttpServerInventoryViewConfig config = TestHelper.getJsonMapper().readValue(json, HttpServerInventoryViewConfig.class); + HttpServerInventoryViewConfig config = TestHelper.makeJsonMapper().readValue(json, HttpServerInventoryViewConfig.class); Assert.assertEquals(TimeUnit.MINUTES.toMillis(4), config.getServerTimeout()); Assert.assertEquals(TimeUnit.MINUTES.toMillis(1), config.getServerUnstabilityTimeout()); @@ -50,7 +50,7 @@ public void testDeserializationWithNonDefaults() throws Exception + " \"numThreads\": 7\n" + "}"; - HttpServerInventoryViewConfig config = TestHelper.getJsonMapper().readValue(json, HttpServerInventoryViewConfig.class); + HttpServerInventoryViewConfig config = TestHelper.makeJsonMapper().readValue(json, HttpServerInventoryViewConfig.class); Assert.assertEquals(TimeUnit.MINUTES.toMillis(2), config.getServerTimeout()); Assert.assertEquals(TimeUnit.MINUTES.toMillis(3), config.getServerUnstabilityTimeout()); diff --git a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java index 81d27a50044f..18476dd119aa 100644 --- a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java @@ -33,9 +33,9 @@ import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.RE; +import io.druid.segment.TestHelper; import io.druid.server.DruidNode; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.SegmentChangeRequestDrop; @@ -70,7 +70,7 @@ public class HttpServerInventoryViewTest @Test(timeout = 10000) public void testSimple() throws Exception { - ObjectMapper jsonMapper = new DefaultObjectMapper(); + ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index 7e6fad72b609..c66aebe13c51 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -35,11 +35,11 @@ import io.druid.client.ServerView; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.announcement.Announcer; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Comparators; +import io.druid.segment.TestHelper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.CuratorDataSegmentServerAnnouncer; import io.druid.server.coordination.DataSegmentServerAnnouncer; @@ -114,7 +114,7 @@ public void setUp() throws Exception cf.blockUntilConnected(); cf.create().creatingParentsIfNeeded().forPath(testBasePath); - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); announcer = new Announcer( cf, diff --git a/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java b/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java index 284c456d5f17..24f997e38666 100644 --- a/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java +++ b/server/src/test/java/io/druid/client/client/ImmutableSegmentLoadInfoTest.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; import io.druid.client.ImmutableSegmentLoadInfo; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; @@ -35,7 +35,7 @@ public class ImmutableSegmentLoadInfoTest { - private final ObjectMapper mapper = new DefaultObjectMapper(); + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); @Test public void testSerde() throws IOException diff --git a/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java b/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java index f2f13c41adf0..e7a82ec86825 100644 --- a/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java +++ b/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java @@ -39,7 +39,7 @@ public void testSerde() throws Exception 1 ); - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); DruidService actual = mapper.readValue( mapper.writeValueAsString(expected), DruidService.class diff --git a/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java b/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java index 69e5ae6930e2..e4bbda5285ac 100644 --- a/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java +++ b/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java @@ -35,7 +35,7 @@ public void testSerde() throws Exception "tier" ); - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); DruidService actual = mapper.readValue( mapper.writeValueAsString(expected), DruidService.class diff --git a/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java b/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java index fd860cb568a0..abdc72b1c422 100644 --- a/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java +++ b/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java @@ -37,7 +37,7 @@ public void testSerde() throws Exception "v1" ); - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); DruidService actual = mapper.readValue( mapper.writeValueAsString(expected), DruidService.class diff --git a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 19d8a665b554..3eddedff0a45 100644 --- a/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -26,9 +26,9 @@ import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.ObjectMetadata; import io.druid.indexing.overlord.SegmentPublishResult; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.segment.TestHelper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NoneShardSpec; @@ -53,7 +53,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final ObjectMapper mapper = new DefaultObjectMapper(); + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); private final DataSegment defaultSegment = new DataSegment( "fooDataSource", Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), diff --git a/server/src/test/java/io/druid/metadata/MetadataSegmentManagerTest.java b/server/src/test/java/io/druid/metadata/MetadataSegmentManagerTest.java index d4f4439d2af7..82252935b4b7 100644 --- a/server/src/test/java/io/druid/metadata/MetadataSegmentManagerTest.java +++ b/server/src/test/java/io/druid/metadata/MetadataSegmentManagerTest.java @@ -26,9 +26,9 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.metamx.emitter.EmittingLogger; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.segment.TestHelper; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -48,7 +48,7 @@ public class MetadataSegmentManagerTest private SQLMetadataSegmentManager manager; private SQLMetadataSegmentPublisher publisher; - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private final DataSegment segment1 = new DataSegment( "wikipedia", diff --git a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java index bfa8bca052aa..3fc085623ffa 100644 --- a/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java +++ b/server/src/test/java/io/druid/query/lookup/LookupSnapshotTakerTest.java @@ -43,7 +43,7 @@ public class LookupSnapshotTakerTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private final ObjectMapper mapper = TestHelper.getJsonMapper(); + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); private LookupSnapshotTaker lookupSnapshotTaker; diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index 3061bfb177c8..65df40f60938 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -58,7 +58,7 @@ public class DataSchemaTest @Rule public ExpectedException expectedException = ExpectedException.none(); - private final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @Test public void testDefaultExclusions() throws Exception diff --git a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java index aed2d22cb345..12878424309c 100644 --- a/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java +++ b/server/src/test/java/io/druid/segment/indexing/RealtimeTuningConfigTest.java @@ -53,7 +53,7 @@ public void testSerdeWithDefaults() throws Exception { String jsonStr = "{\"type\":\"realtime\"}"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( @@ -95,7 +95,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"alertTimeout\": 70\n" + "}"; - ObjectMapper mapper = TestHelper.getJsonMapper(); + ObjectMapper mapper = TestHelper.makeJsonMapper(); RealtimeTuningConfig config = (RealtimeTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java index ec72561aeb00..086763978610 100644 --- a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -196,7 +196,7 @@ public void testSerde() throws Exception ) ); - final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); Assert.assertEquals( transformSpec, jsonMapper.readValue(jsonMapper.writeValueAsString(transformSpec), TransformSpec.class) diff --git a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentFinderTest.java b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentFinderTest.java index d8eb28c10a79..78b48a0ab6aa 100644 --- a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentFinderTest.java +++ b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentFinderTest.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.segment.TestHelper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.apache.commons.io.FileUtils; @@ -44,63 +44,48 @@ public class LocalDataSegmentFinderTest { - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static final ObjectMapper mapper = TestHelper.makeJsonMapper(); private static final String DESCRIPTOR_JSON = "descriptor.json"; private static final String INDEX_ZIP = "index.zip"; - private static final DataSegment SEGMENT_1 = DataSegment.builder() - .dataSource("wikipedia") - .interval( - Intervals.of( - "2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z" - ) - ) - .version("2015-10-21T22:07:57.074Z") - .loadSpec( - ImmutableMap.of( - "type", - "local", - "path", - "/tmp/somewhere/index.zip" - ) - ) - .dimensions(ImmutableList.of("language", "page")) - .metrics(ImmutableList.of("count")) - .build(); - - private static final DataSegment SEGMENT_2 = DataSegment.builder(SEGMENT_1) - .interval( - Intervals.of( - "2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z" - ) - ) - .build(); - - private static final DataSegment SEGMENT_3 = DataSegment.builder(SEGMENT_1) - .interval( - Intervals.of( - "2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" - ) - ) - .version("2015-10-22T22:07:57.074Z") - .build(); - - private static final DataSegment SEGMENT_4_0 = DataSegment.builder(SEGMENT_1) - .interval( - Intervals.of( - "2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" - ) - ) - .shardSpec(new NumberedShardSpec(0, 2)) - .build(); - - private static final DataSegment SEGMENT_4_1 = DataSegment.builder(SEGMENT_1) - .interval( - Intervals.of( - "2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z" - ) - ) - .shardSpec(new NumberedShardSpec(1, 2)) - .build(); + private static final DataSegment SEGMENT_1 = DataSegment + .builder() + .dataSource("wikipedia") + .interval(Intervals.of("2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z")) + .version("2015-10-21T22:07:57.074Z") + .loadSpec( + ImmutableMap.of( + "type", + "local", + "path", + "/tmp/somewhere/index.zip" + ) + ) + .dimensions(ImmutableList.of("language", "page")) + .metrics(ImmutableList.of("count")) + .build(); + + private static final DataSegment SEGMENT_2 = DataSegment + .builder(SEGMENT_1) + .interval(Intervals.of("2013-09-01T00:00:00.000Z/2013-09-02T00:00:00.000Z")) + .build(); + + private static final DataSegment SEGMENT_3 = DataSegment + .builder(SEGMENT_1) + .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z")) + .version("2015-10-22T22:07:57.074Z") + .build(); + + private static final DataSegment SEGMENT_4_0 = DataSegment + .builder(SEGMENT_1) + .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z")) + .shardSpec(new NumberedShardSpec(0, 2)) + .build(); + + private static final DataSegment SEGMENT_4_1 = DataSegment + .builder(SEGMENT_1) + .interval(Intervals.of("2013-09-02T00:00:00.000Z/2013-09-03T00:00:00.000Z")) + .shardSpec(new NumberedShardSpec(1, 2)) + .build(); @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -234,18 +219,4 @@ public void testFindSegmentsFail() throws SegmentLoadingException final LocalDataSegmentFinder localDataSegmentFinder = new LocalDataSegmentFinder(mapper); localDataSegmentFinder.findSegments(dataSourceDir.getAbsolutePath(), false); } - - @Test(expected = SegmentLoadingException.class) - public void testFindSegmentsFail2() throws SegmentLoadingException - { - // will fail to desierialize descriptor.json because DefaultObjectMapper doesn't recognize NumberedShardSpec - final LocalDataSegmentFinder localDataSegmentFinder = new LocalDataSegmentFinder(new DefaultObjectMapper()); - try { - localDataSegmentFinder.findSegments(dataSourceDir.getAbsolutePath(), false); - } - catch (SegmentLoadingException e) { - Assert.assertTrue(e.getCause() instanceof IOException); - throw e; - } - } } diff --git a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java index efd5633747af..2309735424d5 100644 --- a/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java +++ b/server/src/test/java/io/druid/segment/loading/LocalDataSegmentPusherTest.java @@ -23,9 +23,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.Files; import com.google.common.primitives.Ints; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; +import io.druid.segment.TestHelper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; @@ -66,7 +66,7 @@ public void setUp() throws IOException { config = new LocalDataSegmentPusherConfig(); config.storageDirectory = temporaryFolder.newFolder(); - localDataSegmentPusher = new LocalDataSegmentPusher(config, new DefaultObjectMapper()); + localDataSegmentPusher = new LocalDataSegmentPusher(config, TestHelper.makeJsonMapper()); dataSegmentFiles = temporaryFolder.newFolder(); Files.asByteSink(new File(dataSegmentFiles, "version.bin")).write(Ints.toByteArray(0x9)); } diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index ec023fb108bf..d568086319c1 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -115,7 +115,7 @@ public void testSerde() throws Exception TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()), MapCache.create(0), NO_CACHE_CONFIG, - TestHelper.getJsonMapper() + TestHelper.makeJsonMapper() ), null diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java index 4a909d36a806..c0fcc57a7b96 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/CustomVersioningPolicyTest.java @@ -19,7 +19,7 @@ package io.druid.segment.realtime.plumber; -import io.druid.TestUtil; +import io.druid.server.ServerTestHelper; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; @@ -37,8 +37,8 @@ public void testSerialization() throws Exception CustomVersioningPolicy policy = new CustomVersioningPolicy(version); - CustomVersioningPolicy serialized = TestUtil.MAPPER.readValue( - TestUtil.MAPPER.writeValueAsBytes(policy), + CustomVersioningPolicy serialized = ServerTestHelper.MAPPER.readValue( + ServerTestHelper.MAPPER.writeValueAsBytes(policy), CustomVersioningPolicy.class ); 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 efe1dbdef578..cfddca4aa90e 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 @@ -226,7 +226,7 @@ public void setUp() throws Exception TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, - TestHelper.getJsonMapper() + TestHelper.makeJsonMapper() ); metrics = new FireDepartmentMetrics(); diff --git a/server/src/test/java/io/druid/TestUtil.java b/server/src/test/java/io/druid/server/ServerTestHelper.java similarity index 67% rename from server/src/test/java/io/druid/TestUtil.java rename to server/src/test/java/io/druid/server/ServerTestHelper.java index ac3ebe80f6e8..c3fac4a80f54 100644 --- a/server/src/test/java/io/druid/TestUtil.java +++ b/server/src/test/java/io/druid/server/ServerTestHelper.java @@ -17,23 +17,18 @@ * under the License. */ -package io.druid; +package io.druid.server; -import com.fasterxml.jackson.databind.BeanProperty; -import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; - import io.druid.guice.ServerModule; import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.ISE; +import io.druid.timeline.DataSegment; import java.util.List; -/** - */ -public class TestUtil +public class ServerTestHelper { public static final ObjectMapper MAPPER = new DefaultObjectMapper(); @@ -43,19 +38,9 @@ public class TestUtil MAPPER.registerModule(module); } MAPPER.setInjectableValues( - new InjectableValues() - { - @Override - public Object findInjectableValue( - Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance - ) - { - if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { - return TestUtil.MAPPER; - } - throw new ISE("No Injectable value found"); - } - } + new InjectableValues.Std() + .addValue(ObjectMapper.class.getName(), MAPPER) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) ); } } diff --git a/server/src/test/java/io/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/io/druid/server/coordination/SegmentLoadDropHandlerTest.java index 122c83bf07cd..6239c31e7cd2 100644 --- a/server/src/test/java/io/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/io/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -26,11 +26,11 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.java.util.common.logger.Logger; import io.druid.segment.IndexIO; +import io.druid.segment.TestHelper; import io.druid.segment.loading.CacheTestSegmentLoader; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.SegmentManager; @@ -63,7 +63,7 @@ public class SegmentLoadDropHandlerTest private static final Logger log = new Logger(ZkCoordinatorTest.class); - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", "dummyHost", diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index ce106ff4fdac..1ef643f81a96 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -22,13 +22,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.emitter.EmittingLogger; -import io.druid.TestUtil; import io.druid.curator.CuratorTestBase; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; import io.druid.segment.IndexIO; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.SegmentManager; +import io.druid.server.ServerTestHelper; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; @@ -48,7 +47,7 @@ */ public class ZkCoordinatorTest extends CuratorTestBase { - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper jsonMapper = ServerTestHelper.MAPPER; private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", "dummyHost", @@ -102,7 +101,7 @@ public void testLoadDrop() throws Exception CountDownLatch dropLatch = new CountDownLatch(1); SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - TestUtil.MAPPER, + ServerTestHelper.MAPPER, new SegmentLoaderConfig(), EasyMock.createNiceMock(DataSegmentAnnouncer.class), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), 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 fc2eb94539ad..fc59d658ea95 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 @@ -30,8 +30,8 @@ import com.google.common.util.concurrent.MoreExecutors; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.announcement.Announcer; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; +import io.druid.segment.TestHelper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.SegmentChangeRequestHistory; @@ -91,7 +91,7 @@ public void setUp() throws Exception cf.blockUntilConnected(); cf.create().creatingParentsIfNeeded().forPath(testBasePath); - jsonMapper = new DefaultObjectMapper(); + jsonMapper = TestHelper.makeJsonMapper(); announcer = new Announcer( cf, diff --git a/server/src/test/java/io/druid/server/coordinator/HttpLoadQueuePeonTest.java b/server/src/test/java/io/druid/server/coordinator/HttpLoadQueuePeonTest.java index c3eccbf880f2..885058f141a7 100644 --- a/server/src/test/java/io/druid/server/coordinator/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/io/druid/server/coordinator/HttpLoadQueuePeonTest.java @@ -27,12 +27,12 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.HttpResponseHandler; -import io.druid.TestUtil; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.RE; import io.druid.java.util.common.concurrent.Execs; +import io.druid.server.ServerTestHelper; import io.druid.server.coordination.DataSegmentChangeRequest; import io.druid.server.coordination.SegmentLoadDropHandler; import io.druid.timeline.DataSegment; @@ -82,7 +82,7 @@ public void testSimple() throws Exception HttpLoadQueuePeon httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", - TestUtil.MAPPER, + ServerTestHelper.MAPPER, new TestHttpClient(), new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO) { @Override @@ -190,7 +190,7 @@ public ListenableFuture go( httpResponse.setContent(ChannelBuffers.buffer(0)); httpResponseHandler.handleResponse(httpResponse); try { - List changeRequests = TestUtil.MAPPER.readValue( + List changeRequests = ServerTestHelper.MAPPER.readValue( request.getContent().array(), new TypeReference>() {} ); @@ -198,8 +198,13 @@ public ListenableFuture go( for (DataSegmentChangeRequest cr : changeRequests) { statuses.add(new SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus(cr, SegmentLoadDropHandler.Status.SUCCESS)); } - return (ListenableFuture) Futures.immediateFuture(new ByteArrayInputStream(TestUtil.MAPPER.writerWithType( - HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF).writeValueAsBytes(statuses))); + return (ListenableFuture) Futures.immediateFuture( + new ByteArrayInputStream( + ServerTestHelper.MAPPER + .writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) + .writeValueAsBytes(statuses) + ) + ); } catch (Exception ex) { throw new RE(ex, "Unexpected exception."); diff --git a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java index 529ceac38ace..e889968bb303 100644 --- a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java @@ -24,10 +24,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.java.util.common.concurrent.Execs; import io.druid.curator.CuratorTestBase; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.segment.TestHelper; import io.druid.server.coordination.DataSegmentChangeCallback; import io.druid.server.coordination.DataSegmentChangeHandler; import io.druid.server.coordination.DataSegmentChangeRequest; @@ -54,7 +54,7 @@ public class LoadQueuePeonTest extends CuratorTestBase { private static final String LOAD_QUEUE_PATH = "/druid/loadqueue/localhost:1234"; - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private LoadQueuePeon loadQueuePeon; private PathChildrenCache loadQueueCache; diff --git a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java index 5f17c37bd2b8..484b0523bcd2 100644 --- a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java @@ -34,7 +34,7 @@ */ public class CoordinatorDynamicConfigTest { - private final ObjectMapper mapper = TestHelper.getJsonMapper(); + private final ObjectMapper mapper = TestHelper.makeJsonMapper(); @Test public void testSerde() throws Exception diff --git a/server/src/test/java/io/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/NumberedShardSpecTest.java index d17d0e36e203..944480697183 100644 --- a/server/src/test/java/io/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/NumberedShardSpecTest.java @@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import io.druid.TestUtil; import io.druid.java.util.common.Intervals; +import io.druid.server.ServerTestHelper; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.NumberedShardSpec; @@ -45,8 +45,8 @@ public class NumberedShardSpecTest @Test public void testSerdeRoundTrip() throws Exception { - final ShardSpec spec = TestUtil.MAPPER.readValue( - TestUtil.MAPPER.writeValueAsBytes(new NumberedShardSpec(1, 2)), + final ShardSpec spec = ServerTestHelper.MAPPER.readValue( + ServerTestHelper.MAPPER.writeValueAsBytes(new NumberedShardSpec(1, 2)), ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); @@ -56,7 +56,7 @@ public void testSerdeRoundTrip() throws Exception @Test public void testSerdeBackwardsCompat() throws Exception { - final ShardSpec spec = TestUtil.MAPPER.readValue( + final ShardSpec spec = ServerTestHelper.MAPPER.readValue( "{\"type\": \"numbered\", \"partitions\": 2, \"partitionNum\": 1}", ShardSpec.class ); diff --git a/server/src/test/java/io/druid/client/DataSegmentTest.java b/server/src/test/java/io/druid/timeline/DataSegmentTest.java similarity index 95% rename from server/src/test/java/io/druid/client/DataSegmentTest.java rename to server/src/test/java/io/druid/timeline/DataSegmentTest.java index 682498fd0e1d..e6fca682633a 100644 --- a/server/src/test/java/io/druid/client/DataSegmentTest.java +++ b/server/src/test/java/io/druid/timeline/DataSegmentTest.java @@ -17,8 +17,9 @@ * under the License. */ -package io.druid.client; +package io.druid.timeline; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -29,11 +30,11 @@ import io.druid.java.util.common.Intervals; import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.segment.IndexIO; -import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec; import org.joda.time.Interval; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -48,6 +49,14 @@ public class DataSegmentTest { final ObjectMapper mapper = new DefaultObjectMapper(); + @Before + public void setUp() + { + InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); + mapper.setInjectableValues(injectableValues); + } + @Test public void testV1Serialization() throws Exception { diff --git a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java index 5798083c0454..27e789c4c83a 100644 --- a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java @@ -23,14 +23,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; - -import io.druid.TestUtil; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; - +import io.druid.server.ServerTestHelper; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -44,8 +42,15 @@ public class HashBasedNumberedShardSpecTest public void testSerdeRoundTrip() throws Exception { - final ShardSpec spec = TestUtil.MAPPER.readValue( - TestUtil.MAPPER.writeValueAsBytes(new HashBasedNumberedShardSpec(1, 2, ImmutableList.of("visitor_id"), TestUtil.MAPPER)), + final ShardSpec spec = ServerTestHelper.MAPPER.readValue( + ServerTestHelper.MAPPER.writeValueAsBytes( + new HashBasedNumberedShardSpec( + 1, + 2, + ImmutableList.of("visitor_id"), + ServerTestHelper.MAPPER + ) + ), ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); @@ -56,14 +61,14 @@ public void testSerdeRoundTrip() throws Exception @Test public void testSerdeBackwardsCompat() throws Exception { - final ShardSpec spec = TestUtil.MAPPER.readValue( + final ShardSpec spec = ServerTestHelper.MAPPER.readValue( "{\"type\": \"hashed\", \"partitions\": 2, \"partitionNum\": 1}", ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); - final ShardSpec specWithPartitionDimensions = TestUtil.MAPPER.readValue( + final ShardSpec specWithPartitionDimensions = ServerTestHelper.MAPPER.readValue( "{\"type\": \"hashed\", \"partitions\": 2, \"partitionNum\": 1, \"partitionDimensions\":[\"visitor_id\"]}", ShardSpec.class ); @@ -76,9 +81,9 @@ public void testSerdeBackwardsCompat() throws Exception public void testPartitionChunks() { final List specs = ImmutableList.of( - new HashBasedNumberedShardSpec(0, 3, null, TestUtil.MAPPER), - new HashBasedNumberedShardSpec(1, 3, null, TestUtil.MAPPER), - new HashBasedNumberedShardSpec(2, 3, null, TestUtil.MAPPER) + new HashBasedNumberedShardSpec(0, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(1, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(2, 3, null, ServerTestHelper.MAPPER) ); final List> chunks = Lists.transform( @@ -142,7 +147,7 @@ public void testGetGroupKey() throws Exception 1, 2, ImmutableList.of("visitor_id"), - TestUtil.MAPPER + ServerTestHelper.MAPPER ); final DateTime time = DateTimes.nowUtc(); final InputRow inputRow = new MapBasedInputRow( @@ -152,7 +157,12 @@ public void testGetGroupKey() throws Exception ); Assert.assertEquals(ImmutableList.of(Lists.newArrayList("v1")), shardSpec1.getGroupKey(time.getMillis(), inputRow)); - final HashBasedNumberedShardSpec shardSpec2 = new HashBasedNumberedShardSpec(1, 2, null, TestUtil.MAPPER); + final HashBasedNumberedShardSpec shardSpec2 = new HashBasedNumberedShardSpec( + 1, + 2, + null, + ServerTestHelper.MAPPER + ); Assert.assertEquals(ImmutableList.of( time.getMillis(), ImmutableMap.of( @@ -181,7 +191,7 @@ public HashOverridenShardSpec( int partitions ) { - super(partitionNum, partitions, null, TestUtil.MAPPER); + super(partitionNum, partitions, null, ServerTestHelper.MAPPER); } @Override diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 3bade6247bf3..1d8094237eef 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -58,6 +58,7 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.router.TieredBrokerConfig; import io.druid.sql.guice.SqlModule; +import io.druid.timeline.PruneLoadSpec; import org.eclipse.jetty.server.Server; import java.util.List; @@ -94,6 +95,7 @@ public void configure(Binder binder) ); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8082); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8282); + binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(true); binder.bind(CachingClusteredClient.class).in(LazySingleton.class); binder.bind(BrokerServerView.class).in(LazySingleton.class); diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java index 0445f54db102..ea34420a9050 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlQueryTest.java @@ -31,7 +31,7 @@ public class SqlQueryTest @Test public void testSerde() throws Exception { - final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); final SqlQuery query = new SqlQuery("SELECT 1", SqlQuery.ResultFormat.ARRAY, ImmutableMap.of("useCache", false)); Assert.assertEquals(query, jsonMapper.readValue(jsonMapper.writeValueAsString(query), SqlQuery.class)); } 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 a20f24fe2b58..5db831df0327 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 @@ -211,7 +211,7 @@ public AuthenticationResult createEscalatedAuthenticationResult() @Override public void configure(final Binder binder) { - binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.getJsonMapper()); + binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.makeJsonMapper()); // This Module is just to get a LookupReferencesManager with a usable "lookyloo" lookup. @@ -245,7 +245,7 @@ public void configure(final Binder binder) new ScanQueryRunnerFactory( new ScanQueryQueryToolChest( new ScanQueryConfig(), - new DefaultGenericQueryMetricsFactory(TestHelper.getJsonMapper()) + new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) ), new ScanQueryEngine() ) @@ -254,7 +254,7 @@ public void configure(final Binder binder) SelectQuery.class, new SelectQueryRunnerFactory( new SelectQueryQueryToolChest( - TestHelper.getJsonMapper(), + TestHelper.makeJsonMapper(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), SELECT_CONFIG_SUPPLIER ), From 677e24b76027d833da2987416a5439f474faff7a Mon Sep 17 00:00:00 2001 From: Parag Jain Date: Tue, 12 Dec 2017 13:42:30 -0600 Subject: [PATCH 6/7] prevent NPE from supressing actual exception (#5146) --- .../main/java/io/druid/indexing/kafka/KafkaIndexTask.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 312fd1c8cbca..6a17872d7230 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -883,7 +883,10 @@ public void onFailure(Throwable t) if (chatHandlerProvider.isPresent()) { chatHandlerProvider.get().unregister(getId()); } - publishExecService.shutdownNow(); + + if (publishExecService != null) { + publishExecService.shutdownNow(); + } toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().unannounce(); From 3b4395a16e726664d9e2b7cbebe92aa23c20ed86 Mon Sep 17 00:00:00 2001 From: Jonathan Wei Date: Tue, 12 Dec 2017 17:14:44 -0800 Subject: [PATCH 7/7] Use deprecated-2017Q4 group for Travis (#5155) * Use deprecated-2017Q4 group for Travis * Add comment --- .travis.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.travis.yml b/.travis.yml index 6c5454171bfb..4fb64a7bf9af 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,6 +1,11 @@ language: java +# On 12-12-2017, Travis updated their trusty image, which caused integration tests to fail. +# The group: config instructs Travis to use the previous trusty image. +# Please see https://github.com/druid-io/druid/pull/5155 for more information. +sudo: required dist: trusty +group: deprecated-2017Q4 jdk: - oraclejdk8