From e10025c84172ba448b6ac834e5fe784eb6fe5c84 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 01/14] filter out unneeded segments for TimeBoundary query --- .../druid/client/CachingClusteredClient.java | 31 ++++++++++++ .../metamx/druid/query/QueryToolChest.java | 26 ++++++---- .../group/GroupByQueryQueryToolChest.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 2 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../TimeBoundaryQueryQueryToolChest.java | 49 ++++++++++++++++++- .../TimeseriesQueryQueryToolChest.java | 2 +- .../com/metamx/druid/utils/JodaUtils.java | 14 ++++++ .../druid/coordination/ServerManagerTest.java | 2 +- 9 files changed, 115 insertions(+), 15 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 163f1986a53..a07478ab751 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,6 +20,7 @@ package com.metamx.druid.client; import com.google.common.base.Function; +import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -58,6 +59,7 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -152,6 +154,35 @@ public class CachingClusteredClient implements QueryRunner } } + // Let tool chest filter out unneeded segments + final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( + query, + Iterables.transform( + segments, new Function, SegmentDescriptor>() + { + @Override + public SegmentDescriptor apply( + @Nullable Pair input + ) + { + return input.rhs; + } + } + ) + )); + + // remove unneeded segments from list of segments to query + segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() + { + @Override + public boolean apply( + @Nullable Pair input + ) + { + return filteredSegmentDescriptors.contains(input.rhs); + } + })); + final byte[] queryCacheKey; if(strategy != null) { queryCacheKey = strategy.computeCacheKey(query); diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index bec2170ec92..36847f7f77f 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,19 +20,23 @@ package com.metamx.druid.query; import com.google.common.base.Function; +import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.query.segment.QuerySegmentSpec; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential * maintenance burden when upgrading versions. */ -public interface QueryToolChest> +public abstract class QueryToolChest> { - public QueryRunner mergeResults(QueryRunner runner); + public abstract QueryRunner mergeResults(QueryRunner runner); /** * This method doesn't belong here, but it's here for now just to make it work. @@ -40,11 +44,15 @@ public interface QueryToolChest> * @param seqOfSequences * @return */ - public Sequence mergeSequences(Sequence> seqOfSequences); - public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); - public TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query); - public QueryRunner preMergeQueryDecoration(QueryRunner runner); - public QueryRunner postMergeQueryDecoration(QueryRunner runner); + public abstract Sequence mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract TypeReference getResultTypeReference(); + public abstract CacheStrategy getCacheStrategy(QueryType query); + public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); + public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public Iterable filterSegments(QueryType query, Iterable intervals) { + return intervals; + } } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 9dcf6110322..b93f46a4562 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -53,7 +53,7 @@ import java.util.Properties; /** */ -public class GroupByQueryQueryToolChest implements QueryToolChest +public class GroupByQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 160c23cd958..226fc58969f 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -50,7 +50,7 @@ import java.util.Map; import java.util.Set; -public class SegmentMetadataQueryQueryToolChest implements QueryToolChest +public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index ce3fcc86114..a321c7486c5 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -64,7 +64,7 @@ import java.util.Set; /** */ -public class SearchQueryQueryToolChest implements QueryToolChest, SearchQuery> +public class SearchQueryQueryToolChest extends QueryToolChest, SearchQuery> { private static final byte SEARCH_QUERY = 0x2; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 5ee6c321bbb..b566cd84de3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,8 +21,11 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -33,20 +36,26 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; +import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; /** */ public class TimeBoundaryQueryQueryToolChest - implements QueryToolChest, TimeBoundaryQuery> + extends QueryToolChest, TimeBoundaryQuery> { private static final byte TIMEBOUNDARY_QUERY = 0x3; @@ -57,6 +66,44 @@ public class TimeBoundaryQueryQueryToolChest { }; + + @Override + public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + long minMillis = Long.MAX_VALUE; + long maxMillis = Long.MIN_VALUE; + SegmentDescriptor min = null; + SegmentDescriptor max = null; + + // keep track of all segments in a given shard + Map> segmentGroups = Maps.newHashMap(); + + Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); + for(SegmentDescriptor e : input) { + if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); + + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } + + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; + } + } + } + + return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 9c633507ec5..c5f8c280c9a 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -60,7 +60,7 @@ import java.util.Map; /** */ -public class TimeseriesQueryQueryToolChest implements QueryToolChest, TimeseriesQuery> +public class TimeseriesQueryQueryToolChest extends QueryToolChest, TimeseriesQuery> { private static final byte TIMESERIES_QUERY = 0x0; diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 53b14618001..c7feeadc44c 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -19,12 +19,15 @@ package com.metamx.druid.utils; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Iterator; import java.util.TreeSet; @@ -63,6 +66,17 @@ public class JodaUtils return retVal; } + public static boolean overlaps(final Interval i, Iterable intervals) { + return Iterables.any(intervals, new Predicate() + { + @Override + public boolean apply(@Nullable Interval input) + { + return input.overlaps(i); + } + }); + } + public static DateTime minDateTime(DateTime... times) { if (times == null) { diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 84982d05737..165fb402107 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -391,7 +391,7 @@ public class ServerManagerTest } } - public static class NoopQueryToolChest> implements QueryToolChest + public static class NoopQueryToolChest> extends QueryToolChest { @Override public QueryRunner mergeResults(QueryRunner runner) From e9f546c4cabdf489e17f061d076a341c78312078 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:51:40 -0800 Subject: [PATCH 02/14] add default implementations to QueryToolChest methods --- .../com/metamx/druid/query/QueryToolChest.java | 15 ++++++++++++--- .../group/GroupByQueryQueryToolChest.java | 18 ------------------ .../SegmentMetadataQueryQueryToolChest.java | 12 ------------ .../search/SearchQueryQueryToolChest.java | 6 ------ .../TimeBoundaryQueryQueryToolChest.java | 12 ------------ .../TimeseriesQueryQueryToolChest.java | 6 ------ .../druid/coordination/ServerManagerTest.java | 18 ------------------ 7 files changed, 12 insertions(+), 75 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 36847f7f77f..7c462cc08d8 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -48,9 +48,18 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public abstract CacheStrategy getCacheStrategy(QueryType query); - public abstract QueryRunner preMergeQueryDecoration(QueryRunner runner); - public abstract QueryRunner postMergeQueryDecoration(QueryRunner runner); + + public CacheStrategy getCacheStrategy(QueryType query) { + return null; + } + + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + return runner; + } public Iterable filterSegments(QueryType query, Iterable intervals) { return intervals; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index b93f46a4562..1c66774d3bc 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -176,22 +176,4 @@ public class GroupByQueryQueryToolChest extends QueryToolChest getCacheStrategy(GroupByQuery query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 226fc58969f..2b979117800 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -219,18 +219,6 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - private Ordering getOrdering() { return new Ordering() diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index a321c7486c5..cefc32b5e3d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -262,12 +262,6 @@ public class SearchQueryQueryToolChest extends QueryToolChest> postMergeQueryDecoration(final QueryRunner> runner) - { - return runner; - } - private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b566cd84de3..a4647026e79 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -215,18 +215,6 @@ public class TimeBoundaryQueryQueryToolChest }; } - @Override - public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c5f8c280c9a..538897480b3 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -258,12 +258,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest>(runner, Period.months(1)); } - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 165fb402107..de148a9219c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -422,23 +422,5 @@ public class ServerManagerTest { return new TypeReference(){}; } - - @Override - public CacheStrategy getCacheStrategy(QueryType query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } } From 6b87ef2921cf1dd5a1124d872dc32e5668656fbe Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 16:55:21 -0800 Subject: [PATCH 03/14] assume segment intervals have already been filtered based on query interval --- .../TimeBoundaryQueryQueryToolChest.java | 37 ++++++++----------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a4647026e79..8ad7378faf2 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.timeboundary; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -39,15 +38,12 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -77,27 +73,24 @@ public class TimeBoundaryQueryQueryToolChest // keep track of all segments in a given shard Map> segmentGroups = Maps.newHashMap(); - Iterable condensedIntervals = JodaUtils.condenseIntervals(query.getIntervals()); for(SegmentDescriptor e : input) { - if(Iterables.isEmpty(condensedIntervals) || JodaUtils.overlaps(e.getInterval(), condensedIntervals)) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); + final long start = e.getInterval().getStartMillis(); + final long end = e.getInterval().getEndMillis(); + final String version = e.getVersion(); - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } + if(segmentGroups.containsKey(version)) { + segmentGroups.get(version).add(e); + } else { + segmentGroups.put(version, Sets.newHashSet(e)); + } - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + if(min == null || start < minMillis) { + min = e; + minMillis = start; + } + if(max == null || end > maxMillis) { + max = e; + maxMillis = end; } } From d4009c8c1ce9a101295209e742f4a2de7fe2c9ad Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:40:14 -0800 Subject: [PATCH 04/14] refactor filterSegments to use `TimelineObjectHolder`s --- .../druid/client/CachingClusteredClient.java | 51 +++++-------------- .../metamx/druid/query/QueryToolChest.java | 11 +++- .../TimeBoundaryQueryQueryToolChest.java | 17 ++++--- 3 files changed, 34 insertions(+), 45 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index a07478ab751..20c9ca2fea8 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -139,49 +139,26 @@ public class CachingClusteredClient implements QueryRunner // build set of segments to query Set> segments = Sets.newLinkedHashSet(); + List> serversLookup = Lists.newLinkedList(); + for (Interval interval : rewrittenQuery.getIntervals()) { - List> serversLookup = timeline.lookup(interval); - - for (TimelineObjectHolder holder : serversLookup) { - for (PartitionChunk chunk : holder.getObject()) { - ServerSelector selector = chunk.getObject(); - final SegmentDescriptor descriptor = new SegmentDescriptor( - holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() - ); - - segments.add(Pair.of(selector, descriptor)); - } - } + serversLookup.addAll(timeline.lookup(interval)); } // Let tool chest filter out unneeded segments - final Set filteredSegmentDescriptors = Sets.newLinkedHashSet(toolChest.filterSegments( - query, - Iterables.transform( - segments, new Function, SegmentDescriptor>() - { - @Override - public SegmentDescriptor apply( - @Nullable Pair input - ) - { - return input.rhs; - } - } - ) - )); + final List> filteredServersLookup = + toolChest.filterSegments(query, serversLookup); - // remove unneeded segments from list of segments to query - segments = Sets.newLinkedHashSet(Iterables.filter(segments, new Predicate>() - { - @Override - public boolean apply( - @Nullable Pair input - ) - { - return filteredSegmentDescriptors.contains(input.rhs); + for (TimelineObjectHolder holder : filteredServersLookup) { + for (PartitionChunk chunk : holder.getObject()) { + ServerSelector selector = chunk.getObject(); + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() + ); + + segments.add(Pair.of(selector, descriptor)); } - })); + } final byte[] queryCacheKey; if(strategy != null) { diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 7c462cc08d8..6822e654471 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -23,12 +23,16 @@ import com.google.common.base.Function; import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; import org.joda.time.Interval; +import java.util.List; + /** * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential @@ -61,7 +65,10 @@ public abstract class QueryToolChest filterSegments(QueryType query, Iterable intervals) { - return intervals; + public List> filterSegments( + QueryType query, + List> segments + ) { + return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8ad7378faf2..388a3c6df85 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -29,6 +29,8 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -64,16 +66,19 @@ public class TimeBoundaryQueryQueryToolChest @Override - public Iterable filterSegments(TimeBoundaryQuery query, Iterable input) { + public List> filterSegments( + TimeBoundaryQuery query, + List> input + ) { long minMillis = Long.MAX_VALUE; long maxMillis = Long.MIN_VALUE; - SegmentDescriptor min = null; - SegmentDescriptor max = null; + TimelineObjectHolder min = null; + TimelineObjectHolder max = null; // keep track of all segments in a given shard - Map> segmentGroups = Maps.newHashMap(); + Map>> segmentGroups = Maps.newHashMap(); - for(SegmentDescriptor e : input) { + for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); final String version = e.getVersion(); @@ -94,7 +99,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion())); + return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); } @Override From 633cccce6918cece6ebdb9ed2174c9a419dd9872 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 17:02:33 -0800 Subject: [PATCH 05/14] bump up to 0.3.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ec6db24d01c..aa77ea6894b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT druid druid From 7ab2e0552aa954b973d470bf39eb7c23274fc7e9 Mon Sep 17 00:00:00 2001 From: xvrl Date: Mon, 11 Feb 2013 18:42:43 -0800 Subject: [PATCH 06/14] cleanup imports --- .../java/com/metamx/druid/client/CachingClusteredClient.java | 2 -- .../src/main/java/com/metamx/druid/query/QueryToolChest.java | 4 ---- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 3 files changed, 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 20c9ca2fea8..ad08ede04eb 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -20,7 +20,6 @@ package com.metamx.druid.client; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -59,7 +58,6 @@ import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 6822e654471..701a4512403 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -20,16 +20,12 @@ package com.metamx.druid.query; import com.google.common.base.Function; -import com.metamx.common.Pair; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; -import com.metamx.druid.query.segment.QuerySegmentSpec; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceMetricEvent; import org.codehaus.jackson.type.TypeReference; -import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 388a3c6df85..b27a6fd5b80 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -37,7 +37,6 @@ import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; From 68cd2e10a7c42150cb9f5acdef9ca66d7c98b959 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:45:26 -0800 Subject: [PATCH 07/14] remove unnecessary segment grouping --- .../TimeBoundaryQueryQueryToolChest.java | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b27a6fd5b80..8e41f1f6eb8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -74,19 +74,9 @@ public class TimeBoundaryQueryQueryToolChest TimelineObjectHolder min = null; TimelineObjectHolder max = null; - // keep track of all segments in a given shard - Map>> segmentGroups = Maps.newHashMap(); - for(TimelineObjectHolder e : input) { final long start = e.getInterval().getStartMillis(); final long end = e.getInterval().getEndMillis(); - final String version = e.getVersion(); - - if(segmentGroups.containsKey(version)) { - segmentGroups.get(version).add(e); - } else { - segmentGroups.put(version, Sets.newHashSet(e)); - } if(min == null || start < minMillis) { min = e; @@ -98,7 +88,7 @@ public class TimeBoundaryQueryQueryToolChest } } - return Lists.newArrayList(Sets.union(segmentGroups.get(min.getVersion()), segmentGroups.get(max.getVersion()))); + return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); } @Override From 07b0b4611bae8063698e220372678e0907b155e4 Mon Sep 17 00:00:00 2001 From: xvrl Date: Tue, 12 Feb 2013 09:52:21 -0800 Subject: [PATCH 08/14] bump up to 0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 6 +++--- examples/pom.xml | 2 +- index-common/pom.xml | 4 ++-- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 82c917ebecb..36bb84de0d4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 8d058d643cf..350828b9316 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9b2567dc753..88994a529d5 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -66,4 +66,4 @@ - \ No newline at end of file + diff --git a/examples/pom.xml b/examples/pom.xml index fa195a3f0f4..b24a0f7dd42 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 71a7d3192cc..9d17c6b2c53 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT @@ -94,4 +94,4 @@ - \ No newline at end of file + diff --git a/indexer/pom.xml b/indexer/pom.xml index 413582985a2..d38a3a17769 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index c24a262529d..980aa96d5f0 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/realtime/pom.xml b/realtime/pom.xml index 8a71c169fd4..a8c50128f2b 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bf0a71f4d75..544108cef5f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From c5b78e75cfe80b594ad5eb09db94ec830fa1e72b Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 12 Feb 2013 20:58:12 -0600 Subject: [PATCH 09/14] 1) Move up to Jackson 2.1 2) Some bugs were fixed, but they ended up getting mixed in with the Jackson upgrade :( --- client/pom.xml | 19 +++--- .../main/java/com/metamx/druid/BaseNode.java | 3 +- .../main/java/com/metamx/druid/BaseQuery.java | 3 +- .../src/main/java/com/metamx/druid/Query.java | 6 +- .../java/com/metamx/druid/QueryableNode.java | 9 ++- .../com/metamx/druid/RegisteringNode.java | 3 +- .../metamx/druid/client/BrokerServerView.java | 3 +- .../druid/client/CachingClusteredClient.java | 6 +- .../com/metamx/druid/client/DataSegment.java | 12 ++-- .../druid/client/DirectDruidClient.java | 20 ++++--- .../metamx/druid/client/DruidDataSource.java | 3 +- .../com/metamx/druid/client/DruidServer.java | 6 +- .../com/metamx/druid/client/ZKPhoneBook.java | 3 +- .../com/metamx/druid/http/BrokerNode.java | 6 +- .../druid/http/ClientServletModule.java | 6 +- .../com/metamx/druid/http/QueryServlet.java | 6 +- .../druid/initialization/Initialization.java | 3 +- .../metamx/druid/merge/ClientAppendQuery.java | 6 +- .../druid/merge/ClientDefaultMergeQuery.java | 6 +- .../metamx/druid/merge/ClientDeleteQuery.java | 6 +- .../metamx/druid/merge/ClientMergeQuery.java | 8 ++- .../com/metamx/druid/query/CacheStrategy.java | 3 +- .../metamx/druid/query/QueryToolChest.java | 3 +- .../query/dimension/DefaultDimensionSpec.java | 6 +- .../druid/query/dimension/DimensionSpec.java | 6 +- .../dimension/ExtractionDimensionSpec.java | 6 +- .../query/dimension/LegacyDimensionSpec.java | 3 +- .../query/extraction/DimExtractionFn.java | 5 +- .../extraction/PartialDimExtractionFn.java | 7 ++- .../extraction/RegexDimExtractionFn.java | 7 ++- .../SearchQuerySpecDimExtractionFn.java | 6 +- .../query/extraction/TimeDimExtractionFn.java | 6 +- .../druid/query/filter/AndDimFilter.java | 6 +- .../metamx/druid/query/filter/DimFilter.java | 5 +- .../query/filter/ExtractionDimFilter.java | 6 +- .../druid/query/filter/NotDimFilter.java | 7 ++- .../druid/query/filter/OrDimFilter.java | 6 +- .../druid/query/filter/RegexDimFilter.java | 6 +- .../query/filter/SearchQueryDimFilter.java | 3 +- .../druid/query/filter/SelectorDimFilter.java | 7 ++- .../druid/query/group/GroupByQuery.java | 6 +- .../group/GroupByQueryQueryToolChest.java | 3 +- .../druid/query/metadata/ColumnAnalysis.java | 6 +- .../query/metadata/ColumnIncluderator.java | 5 +- .../metadata/ListColumnIncluderator.java | 6 +- .../druid/query/metadata/SegmentAnalysis.java | 6 +- .../query/metadata/SegmentMetadataQuery.java | 3 +- .../SegmentMetadataQueryQueryToolChest.java | 3 +- .../query/search/FragmentSearchQuerySpec.java | 6 +- .../InsensitiveContainsSearchQuerySpec.java | 7 ++- .../search/LexicographicSearchSortSpec.java | 4 +- .../metamx/druid/query/search/SearchHit.java | 7 ++- .../druid/query/search/SearchQuery.java | 6 +- .../search/SearchQueryQueryToolChest.java | 3 +- .../druid/query/search/SearchQuerySpec.java | 6 +- .../druid/query/search/SearchSortSpec.java | 7 ++- .../query/segment/LegacySegmentSpec.java | 3 +- .../segment/MultipleIntervalSegmentSpec.java | 6 +- .../segment/MultipleSpecificSegmentSpec.java | 6 +- .../druid/query/segment/QuerySegmentSpec.java | 8 ++- .../query/segment/SegmentDescriptor.java | 6 +- .../query/timeboundary/TimeBoundaryQuery.java | 6 +- .../TimeBoundaryQueryQueryToolChest.java | 3 +- .../query/timeseries/TimeseriesQuery.java | 11 ++-- .../TimeseriesQueryQueryToolChest.java | 3 +- .../result/BySegmentResultValueClass.java | 3 +- .../result/BySegmentSearchResultValue.java | 6 +- .../druid/result/MetricValueExtractor.java | 7 ++- .../java/com/metamx/druid/result/Result.java | 8 ++- .../druid/result/SearchResultValue.java | 6 +- .../druid/result/TimeBoundaryResultValue.java | 6 +- .../druid/result/TimeseriesResultValue.java | 4 +- .../metamx/druid/shard/LinearShardSpec.java | 3 +- .../com/metamx/druid/shard/NoneShardSpec.java | 2 +- .../com/metamx/druid/shard/ShardSpec.java | 8 ++- .../druid/shard/SingleDimensionShardSpec.java | 5 +- .../metamx/druid/client/DataSegmentTest.java | 6 +- .../druid/query/QuerySegmentSpecTest.java | 3 +- common/pom.xml | 15 +++-- .../com/metamx/druid/DurationGranularity.java | 6 +- .../com/metamx/druid/PeriodGranularity.java | 6 +- .../com/metamx/druid/QueryGranularity.java | 11 ++-- .../druid/aggregation/AggregatorFactory.java | 8 ++- .../aggregation/CountAggregatorFactory.java | 6 +- .../DoubleSumAggregatorFactory.java | 6 +- .../metamx/druid/aggregation/Histogram.java | 3 +- .../HistogramAggregatorFactory.java | 6 +- .../druid/aggregation/HistogramVisual.java | 9 ++- .../JavaScriptAggregatorFactory.java | 6 +- .../aggregation/LongSumAggregatorFactory.java | 6 +- .../aggregation/MaxAggregatorFactory.java | 6 +- .../aggregation/MinAggregatorFactory.java | 6 +- .../post/ArithmeticPostAggregator.java | 6 +- .../post/ConstantPostAggregator.java | 6 +- .../post/FieldAccessPostAggregator.java | 6 +- .../aggregation/post/PostAggregator.java | 7 ++- .../metamx/druid/db/DbConnectorConfig.java | 3 +- .../com/metamx/druid/input/MapBasedRow.java | 6 +- .../main/java/com/metamx/druid/input/Row.java | 7 ++- .../jackson/CommaListJoinDeserializer.java | 13 ++-- .../jackson/CommaListJoinSerializer.java | 14 +++-- .../druid/jackson/DefaultObjectMapper.java | 60 +++++++++++++------ .../com/metamx/druid/jackson/JodaStuff.java | 28 +++++---- .../metamx/druid/QueryGranularityTest.java | 3 +- .../metamx/druid/histogram/HistogramTest.java | 3 +- .../jackson/DefaultObjectMapperTest.java | 40 +++++++++++++ examples/rand/pom.xml | 19 +++--- .../druid/examples/RandomFirehoseFactory.java | 9 ++- .../examples/RealtimeStandaloneMain.java | 3 +- examples/twitter/pom.xml | 19 +++--- .../examples/RealtimeStandaloneMain.java | 3 +- .../TwitterSpritzerFirehoseFactory.java | 9 ++- index-common/pom.xml | 7 ++- .../index/column/ColumnCapabilitiesImpl.java | 3 +- .../druid/index/column/ColumnDescriptor.java | 6 +- .../druid/index/serde/ColumnPartSerde.java | 6 +- .../index/serde/ComplexColumnPartSerde.java | 6 +- .../DictionaryEncodedColumnPartSerde.java | 6 +- .../serde/FloatGenericColumnPartSerde.java | 6 +- .../serde/LongGenericColumnPartSerde.java | 6 +- .../com/metamx/druid/index/v1/IndexIO.java | 3 +- .../druid/index/v1/serde/Registererer.java | 3 +- .../druid/indexer/data/CSVDataSpec.java | 6 +- .../metamx/druid/indexer/data/DataSpec.java | 6 +- .../druid/indexer/data/DelimitedDataSpec.java | 6 +- .../druid/indexer/data/JSONDataSpec.java | 3 +- .../indexer/data/StringInputRowParser.java | 6 +- .../druid/indexer/data/TimestampSpec.java | 6 +- .../indexer/data/ToLowercaseDataSpec.java | 3 +- indexer/pom.xml | 11 ++-- .../metamx/druid/indexer/DbUpdaterJob.java | 3 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../druid/indexer/HadoopDruidIndexer.java | 2 +- .../indexer/HadoopDruidIndexerConfig.java | 59 +++++++++--------- .../druid/indexer/HadoopDruidIndexerNode.java | 4 +- .../druid/indexer/HadoopyShardSpec.java | 6 +- .../java/com/metamx/druid/indexer/Utils.java | 6 +- .../granularity/ArbitraryGranularitySpec.java | 6 +- .../indexer/granularity/GranularitySpec.java | 6 +- .../granularity/UniformGranularitySpec.java | 6 +- .../indexer/partitions/PartitionsSpec.java | 9 ++- .../path/GranularUnprocessedPathSpec.java | 3 +- .../indexer/path/GranularityPathSpec.java | 3 +- .../metamx/druid/indexer/path/PathSpec.java | 8 ++- .../druid/indexer/path/StaticPathSpec.java | 3 +- .../druid/indexer/rollup/DataRollupSpec.java | 3 +- .../indexer/updater/DbUpdaterJobSpec.java | 3 +- .../druid/indexer/updater/UpdaterJobSpec.java | 5 +- .../indexer/updater/ZkUpdaterJobSpec.java | 3 +- .../indexer/HadoopDruidIndexerConfigTest.java | 7 ++- .../granularity/ArbitraryGranularityTest.java | 3 +- .../granularity/UniformGranularityTest.java | 3 +- merger/pom.xml | 15 ++--- .../druid/merger/common/TaskHolder.java | 6 +- .../druid/merger/common/TaskStatus.java | 6 +- .../druid/merger/common/TaskToolbox.java | 3 +- .../common/index/StaticS3FirehoseFactory.java | 12 ++-- .../common/index/YeOldePlumberSchool.java | 12 ++-- .../merger/common/task/AbstractTask.java | 6 +- .../druid/merger/common/task/AppendTask.java | 6 +- .../merger/common/task/DefaultMergeTask.java | 6 +- .../druid/merger/common/task/DeleteTask.java | 8 ++- .../task/IndexDeterminePartitionsTask.java | 9 ++- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/IndexTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 9 ++- .../metamx/druid/merger/common/task/Task.java | 6 +- .../merger/common/task/V8toV9UpgradeTask.java | 3 +- .../merger/coordinator/DbTaskStorage.java | 3 +- .../coordinator/MergerDBCoordinator.java | 3 +- .../merger/coordinator/RemoteTaskRunner.java | 3 +- .../druid/merger/coordinator/TaskContext.java | 6 +- .../merger/coordinator/WorkerWrapper.java | 3 +- .../config/IndexerDbConnectorConfig.java | 3 +- .../http/IndexerCoordinatorNode.java | 6 +- .../http/IndexerCoordinatorServletModule.java | 6 +- .../scaling/EC2AutoScalingStrategy.java | 3 +- .../merger/coordinator/setup/EC2NodeData.java | 7 ++- .../coordinator/setup/GalaxyUserData.java | 5 +- .../coordinator/setup/WorkerSetupData.java | 7 ++- .../coordinator/setup/WorkerSetupManager.java | 3 +- .../druid/merger/worker/TaskMonitor.java | 2 +- .../metamx/druid/merger/worker/Worker.java | 6 +- .../worker/WorkerCuratorCoordinator.java | 3 +- .../druid/merger/worker/http/WorkerNode.java | 6 +- .../coordinator/RemoteTaskRunnerTest.java | 14 +++-- pom.xml | 43 ++++++++----- realtime/pom.xml | 15 ++--- .../metamx/druid/realtime/FireDepartment.java | 7 ++- .../druid/realtime/FireDepartmentConfig.java | 3 +- .../druid/realtime/FirehoseFactory.java | 7 ++- .../druid/realtime/KafkaFirehoseFactory.java | 6 +- .../druid/realtime/MetadataUpdater.java | 3 +- .../metamx/druid/realtime/PlumberSchool.java | 5 +- .../metamx/druid/realtime/RealtimeNode.java | 18 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 +++-- .../druid/realtime/S3SegmentPusher.java | 3 +- .../com/metamx/druid/realtime/Schema.java | 6 +- server/pom.xml | 27 +++++---- .../src/main/java/com/metamx/TsvToJson.java | 3 +- .../java/com/metamx/druid/BaseServerNode.java | 3 +- .../DataSegmentChangeRequest.java | 8 ++- .../SegmentChangeRequestDrop.java | 9 ++- .../SegmentChangeRequestLoad.java | 9 ++- .../druid/coordination/ZkCoordinator.java | 3 +- .../metamx/druid/db/DatabaseRuleManager.java | 6 +- .../druid/db/DatabaseSegmentManager.java | 3 +- .../com/metamx/druid/http/ComputeNode.java | 6 +- .../com/metamx/druid/http/MasterMain.java | 3 +- .../druid/http/MasterServletModule.java | 6 +- .../com/metamx/druid/http/SegmentToDrop.java | 5 +- .../com/metamx/druid/http/SegmentToMove.java | 5 +- .../druid/index/brita/SearchQueryFilter.java | 6 +- .../druid/index/v1/IndexGranularity.java | 3 +- .../metamx/druid/loading/S3SegmentPusher.java | 3 +- .../com/metamx/druid/master/DruidMaster.java | 3 +- .../master/DruidMasterSegmentMerger.java | 3 +- .../metamx/druid/master/HttpMergerClient.java | 3 +- .../druid/master/rules/IntervalDropRule.java | 6 +- .../druid/master/rules/IntervalLoadRule.java | 6 +- .../druid/master/rules/PeriodDropRule.java | 6 +- .../druid/master/rules/PeriodLoadRule.java | 6 +- .../com/metamx/druid/master/rules/Rule.java | 6 +- .../SegmentChangeRequestDropTest.java | 6 +- .../SegmentChangeRequestLoadTest.java | 6 +- .../druid/coordination/ServerManagerTest.java | 3 +- .../druid/coordination/ZkCoordinatorTest.java | 3 +- 227 files changed, 1029 insertions(+), 542 deletions(-) create mode 100644 common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java diff --git a/client/pom.xml b/client/pom.xml index 36bb84de0d4..91674a214cc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -106,20 +107,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject diff --git a/client/src/main/java/com/metamx/druid/BaseNode.java b/client/src/main/java/com/metamx/druid/BaseNode.java index 7c7ed5f0984..8902d14a65f 100644 --- a/client/src/main/java/com/metamx/druid/BaseNode.java +++ b/client/src/main/java/com/metamx/druid/BaseNode.java @@ -19,9 +19,10 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.util.Properties; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java index 76448ed9f17..9f3ae8d30ed 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -19,13 +19,14 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/Query.java b/client/src/main/java/com/metamx/druid/Query.java index 4c4e7f715b4..818400fc262 100644 --- a/client/src/main/java/com/metamx/druid/Query.java +++ b/client/src/main/java/com/metamx/druid/Query.java @@ -19,6 +19,8 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.group.GroupByQuery; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 76e835d4ffe..749b2bd8b40 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -19,6 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -47,9 +50,9 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.metamx.phonebook.PhoneBook; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; -import org.codehaus.jackson.smile.SmileFactory; + + + import org.mortbay.jetty.Server; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/RegisteringNode.java b/client/src/main/java/com/metamx/druid/RegisteringNode.java index 93d8fd7c8cd..1332bbd2a4b 100644 --- a/client/src/main/java/com/metamx/druid/RegisteringNode.java +++ b/client/src/main/java/com/metamx/druid/RegisteringNode.java @@ -19,8 +19,9 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.index.v1.serde.Registererer; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java index e37e0bfeb01..98900e9de41 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -28,7 +29,7 @@ import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.http.client.HttpClient; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.Iterator; diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index ad08ede04eb..6d896d8bf0f 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -53,8 +55,8 @@ import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index 9b29f3507ee..b915f7680cd 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -19,6 +19,10 @@ package com.metamx.druid.client; +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.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -31,10 +35,10 @@ import com.metamx.druid.jackson.CommaListJoinSerializer; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.annotate.JsonDeserialize; -import org.codehaus.jackson.map.annotate.JsonSerialize; + + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java index 62782a41892..7c3e23c1395 100644 --- a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java +++ b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java @@ -19,6 +19,13 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.TypeFactory; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.Closeables; @@ -40,13 +47,12 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.ObjectCodec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.type.TypeFactory; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.JavaType; + + + + + + import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; diff --git a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java index 6f96e8e8672..e112bbf6234 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java +++ b/client/src/main/java/com/metamx/druid/client/DruidDataSource.java @@ -19,8 +19,9 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Collections; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index 6ecfdb16a64..a71d8a15ac7 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -19,10 +19,12 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.HashMap; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java index af9d552d009..1b8fa9df876 100644 --- a/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java +++ b/client/src/main/java/com/metamx/druid/client/ZKPhoneBook.java @@ -19,6 +19,7 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import org.I0Itec.zkclient.IZkChildListener; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.exception.ZkNoNodeException; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import java.io.IOException; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 2a94a00d76d..76e487efb56 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.inject.Guice; @@ -53,8 +55,8 @@ import com.metamx.http.client.HttpClientInit; import com.metamx.metrics.Monitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; diff --git a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java index 7ecc5d7ad81..c5b89f37004 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java +++ b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java @@ -19,13 +19,15 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ClientInventoryManager; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 108863bf7af..9a13c50f45c 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; @@ -31,8 +33,8 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.ObjectWriter; + + import org.joda.time.DateTime; import org.mortbay.jetty.Request; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index cfb2d9302a2..4865208e88e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -19,6 +19,7 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -41,7 +42,7 @@ import com.netflix.curator.x.discovery.ServiceInstance; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Connector; import org.mortbay.jetty.Server; import org.mortbay.jetty.nio.SelectChannelConnector; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java index 441adcffc3b..5a8e3bdb50f 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java @@ -19,9 +19,11 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java index c5cc8045492..4286cd211cb 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java @@ -19,10 +19,12 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java index a0733db0a2a..3acf20a0a43 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java @@ -19,8 +19,10 @@ package com.metamx.druid.merge; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; public class ClientDeleteQuery diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java index 83f269d0db3..3956991ed6b 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java @@ -19,15 +19,17 @@ package com.metamx.druid.merge; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) }) diff --git a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java index f8f5098f6ca..b17e95b86e7 100644 --- a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java +++ b/client/src/main/java/com/metamx/druid/query/CacheStrategy.java @@ -19,10 +19,11 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; -import org.codehaus.jackson.type.TypeReference; + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 701a4512403..f729016032e 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -19,13 +19,14 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java index a936023550f..a3c36476253 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java index 34310ef4c09..bcf5fabf62b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java index fe80c24c94f..6feb011dcd7 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java index 9a4e11a9a5f..b753b04c46b 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.dimension; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java index 0c1ef8b159c..8fe869cb4af 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java index 28ea1ebc37b..f30c4578706 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java index da3a878aa2f..c722b0c52b7 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.extraction; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 57e3bfde16b..7dbc2f3c65d 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java index dc2c2eb36cf..c02797cf088 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.extraction; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.ibm.icu.text.SimpleDateFormat; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.text.ParseException; diff --git a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java index 92696b29c26..6b557fdba27 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java index 9dc819cb8ec..8da47da465c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java index db3376b9de8..21785f06b0c 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.extraction.DimExtractionFn; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java index 8b474d301f3..0753d9f64de 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java index 2a056c8dfd6..9105cfe0412 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java index 519814d8262..6ee15d4b0c9 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java index b579acd87c5..7b1b6b0fd15 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java @@ -19,9 +19,10 @@ package com.metamx.druid.query.filter; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java index c778027d48c..11deaeb28e2 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.filter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index d30af309473..4c1ac740dca 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -33,8 +35,8 @@ import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.LegacySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 1c66774d3bc..51570e30955 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.group; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -43,7 +44,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java index 33b894ca07a..9eb8a46bcdb 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.index.column.ValueType; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java index 90533c4eaca..c6cfb081835 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java index e74661d6822..a7de6b1920e 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java @@ -19,12 +19,14 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java index 1182bfeb9fa..22657e45301 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.metadata; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index 7e0d04c0739..707a0a285b2 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.BaseQuery; import com.metamx.druid.Query; import com.metamx.druid.query.segment.QuerySegmentSpec; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 2b979117800..6ede9c7f247 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.metadata; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -39,7 +40,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index e0a73b4bc3c..b813b7febbf 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 87a6246a5b2..1d7a836a3ab 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java index b7dfddef413..83f81bfc349 100644 --- a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java @@ -19,7 +19,9 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Comparator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java index c1c0727067a..ba896912197 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchHit.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import static com.google.common.base.Preconditions.checkNotNull; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index 4ce3bd72e7f..124d9c5fa62 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -29,8 +31,8 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index cefc32b5e3d..f58f31ebad7 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Joiner; @@ -49,7 +50,7 @@ import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java index 3f634bd3530..dbd0e91c48d 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.query.search; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Predicate; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java index b68f074e2ce..f98cf41f999 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java @@ -19,8 +19,11 @@ package com.metamx.druid.query.search; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java index 6d25c152ab4..f4e62ff77c2 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java index f81126857bc..5432d468ed4 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.Collections; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java index 4193934fbc0..fb259c5e5d5 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.utils.JodaUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java index ca42ce8bc8a..98fc8e7b327 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.query.segment; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.Query; import com.metamx.druid.query.QueryRunner; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.Interval; import java.util.List; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) @JsonSubTypes(value={ @JsonSubTypes.Type(name="intervals", value=MultipleIntervalSegmentSpec.class), @JsonSubTypes.Type(name="segments", value=MultipleSpecificSegmentSpec.class) diff --git a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java index d918309baa6..90d433c1e5d 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.segment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index 071d7695c0f..07fd071d6db 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -19,6 +19,8 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -28,8 +30,8 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 8e41f1f6eb8..86e618bcee8 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeboundary; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.Lists; @@ -40,7 +41,7 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index 490e2531edd..a67caf19924 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -19,6 +19,9 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.BaseQuery; @@ -31,10 +34,10 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeName; + + + + import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 538897480b3..bef34d9a2da 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query.timeseries; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.Lists; @@ -45,7 +46,7 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java index fb1e0df8e22..dfaacb572cc 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java @@ -19,7 +19,8 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java index b324ccbab37..e5a069f17a1 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java index 3baf45c6a67..c99ac0f3ed3 100644 --- a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java +++ b/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java @@ -19,8 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import java.lang.Double;import java.lang.Float;import java.lang.Long;import java.lang.Number;import java.lang.Object;import java.lang.Override;import java.lang.String;import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/Result.java b/client/src/main/java/com/metamx/druid/result/Result.java index 9b055cd4800..f9c30fc8319 100644 --- a/client/src/main/java/com/metamx/druid/result/Result.java +++ b/client/src/main/java/com/metamx/druid/result/Result.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.DateTime; /** diff --git a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java index 40f10ed85ec..c9dd74fc4ec 100644 --- a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/SearchResultValue.java @@ -19,9 +19,11 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.druid.query.search.SearchHit; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import java.util.Iterator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java index 74daa14f376..1b1b4c45547 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java @@ -19,10 +19,12 @@ package com.metamx.druid.result; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.IAE; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonValue; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java index d68a9855395..55892023def 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java +++ b/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java @@ -19,7 +19,9 @@ package com.metamx.druid.result; -import org.codehaus.jackson.annotate.JsonCreator; + + +import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java index 97f22ae6660..ae83287f290 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java @@ -1,9 +1,10 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.LinearPartitionChunk; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java index 4ed756e1daf..9e40f9f703e 100644 --- a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java @@ -22,7 +22,7 @@ package com.metamx.druid.shard; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.SingleElementPartitionChunk; -import org.codehaus.jackson.annotate.JsonTypeName; + import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java index df3eb29004a..2db8f15ed5f 100644 --- a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/ShardSpec.java @@ -19,17 +19,19 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Map; /** * A Marker interface that exists to combine ShardSpec objects together for Jackson */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", include=JsonTypeInfo.As.PROPERTY) @JsonSubTypes({ @JsonSubTypes.Type(name="single", value=SingleDimensionShardSpec.class), @JsonSubTypes.Type(name="none", value=NoneShardSpec.class), diff --git a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java index 7580673b071..0843ec7c61d 100644 --- a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.StringPartitionChunk; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + import java.util.List; import java.util.Map; diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java index 24d5986a8f8..082f66e8aac 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java @@ -19,6 +19,8 @@ package com.metamx.druid.client; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -27,8 +29,8 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java index a05973af729..654ea2e6b79 100644 --- a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java +++ b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.query; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -27,7 +28,7 @@ import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.SegmentDescriptor; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/common/pom.xml b/common/pom.xml index 350828b9316..f334b521775 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-common @@ -72,12 +73,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/DurationGranularity.java b/common/src/main/java/com/metamx/druid/DurationGranularity.java index da485b62e97..b2d5e55be42 100644 --- a/common/src/main/java/com/metamx/druid/DurationGranularity.java +++ b/common/src/main/java/com/metamx/druid/DurationGranularity.java @@ -19,9 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/PeriodGranularity.java b/common/src/main/java/com/metamx/druid/PeriodGranularity.java index 6fb461a5bc9..3c6ca5fd23e 100644 --- a/common/src/main/java/com/metamx/druid/PeriodGranularity.java +++ b/common/src/main/java/com/metamx/druid/PeriodGranularity.java @@ -19,8 +19,10 @@ package com.metamx.druid; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/common/src/main/java/com/metamx/druid/QueryGranularity.java b/common/src/main/java/com/metamx/druid/QueryGranularity.java index 36a4dd01714..5b94c5e1ae2 100644 --- a/common/src/main/java/com/metamx/druid/QueryGranularity.java +++ b/common/src/main/java/com/metamx/druid/QueryGranularity.java @@ -19,14 +19,17 @@ package com.metamx.druid; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.ReadableDuration; -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "period", value = PeriodGranularity.class), @JsonSubTypes.Type(name = "duration", value = DurationGranularity.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java index 00722fcf5eb..c12793972f2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.Comparator; import java.util.List; @@ -36,7 +38,7 @@ import java.util.List; * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how * the data is actually stored and accessed. */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java index 61dcde40449..a901c3f54e2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java index dc8bc9a9842..92a972d0658 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java index bb27e317eb4..a3ec32d4a15 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/com/metamx/druid/aggregation/Histogram.java @@ -19,11 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import org.codehaus.jackson.annotate.JsonValue; + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java index 8902ae56bef..af6d23c32ac 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java @@ -19,13 +19,15 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java index ab4e8836fe1..01c01719854 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java @@ -19,16 +19,19 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; public class HistogramVisual { @JsonProperty final public double[] breaks; - @JsonProperty final public double[] counts; + @JsonProperty + final public double[] counts; // an array of the quantiles including the min. and max. @JsonProperty final public double[] quantiles; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java index 48e5cf776f5..5c257f8cc74 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java @@ -19,14 +19,16 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Joiner; import com.google.common.collect.Lists; import com.google.common.primitives.Doubles; import com.metamx.common.IAE; import com.metamx.druid.processing.FloatMetricSelector; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; import org.mozilla.javascript.Script; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java index 75e59b39e2c..61247e40add 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java index 85a5466ec53..76a57f23066 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java index 49b2ad154c9..2720fa2a9f1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Doubles; import com.metamx.druid.processing.MetricSelectorFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java index cec19d80d78..1166413a3e7 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java @@ -19,11 +19,13 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.IAE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Iterator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java index f1bbb0d8392..f36761f962f 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java @@ -19,9 +19,11 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java index 780c720103d..eb3a531e2d1 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java @@ -19,10 +19,12 @@ package com.metamx.druid.aggregation.post; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.ISE; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java index 5b1ebc60528..66427785fbc 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java @@ -19,8 +19,11 @@ package com.metamx.druid.aggregation.post; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index b6e195cd5b7..808be63267c 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -19,7 +19,8 @@ package com.metamx.druid.db; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; import org.skife.config.Config; /** diff --git a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java index d823762eddd..5449ff2f865 100644 --- a/common/src/main/java/com/metamx/druid/input/MapBasedRow.java +++ b/common/src/main/java/com/metamx/druid/input/MapBasedRow.java @@ -19,13 +19,15 @@ package com.metamx.druid.input; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.common.exception.FormattedException; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/common/src/main/java/com/metamx/druid/input/Row.java b/common/src/main/java/com/metamx/druid/input/Row.java index add4623d498..353d3e52f06 100644 --- a/common/src/main/java/com/metamx/druid/input/Row.java +++ b/common/src/main/java/com/metamx/druid/input/Row.java @@ -19,8 +19,11 @@ package com.metamx.druid.input; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.List; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java index 1932746b3d0..dd18713d797 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinDeserializer.java @@ -19,10 +19,15 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.deser.std.StdScalarDeserializer; + + + + + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer; import java.io.IOException; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java index 3b270ad4aa1..a837c88bbe7 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommaListJoinSerializer.java @@ -19,18 +19,22 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer; import com.google.common.base.Joiner; -import org.codehaus.jackson.JsonGenerationException; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.ser.std.ScalarSerializerBase; + + + + import java.io.IOException; import java.util.List; /** */ -public class CommaListJoinSerializer extends ScalarSerializerBase> +public class CommaListJoinSerializer extends StdScalarSerializer> { private static final Joiner joiner = Joiner.on(","); diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index c80958eff88..84514bd6c4c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -19,26 +19,45 @@ package com.metamx.druid.jackson; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.Version; +import com.fasterxml.jackson.databind.DeserializationConfig; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationConfig; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; -import org.codehaus.jackson.JsonFactory; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.Version; -import org.codehaus.jackson.map.DeserializationConfig; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.JsonDeserializer; -import org.codehaus.jackson.map.JsonSerializer; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.SerializationConfig; -import org.codehaus.jackson.map.SerializerProvider; -import org.codehaus.jackson.map.Serializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + + + + + + + + + + + + + + + import org.joda.time.DateTimeZone; +import org.skife.jdbi.v2.sqlobject.customizers.Mapper; import java.io.IOException; import java.nio.ByteOrder; @@ -153,9 +172,12 @@ public class DefaultObjectMapper extends ObjectMapper ); registerModule(serializerModule); - configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); - configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, false); - configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, false); - configure(SerializationConfig.Feature.INDENT_OUTPUT, false); + configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + configure(MapperFeature.AUTO_DETECT_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_CREATORS, false); + configure(MapperFeature.AUTO_DETECT_FIELDS, false); + configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); + configure(MapperFeature.AUTO_DETECT_SETTERS, false); + configure(SerializationFeature.INDENT_OUTPUT, false); } } diff --git a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java index 50d39378390..f06bee8c76d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java +++ b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java @@ -19,19 +19,21 @@ package com.metamx.druid.jackson; -import org.codehaus.jackson.JsonParser; -import org.codehaus.jackson.JsonProcessingException; -import org.codehaus.jackson.JsonToken; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.KeyDeserializer; -import org.codehaus.jackson.map.deser.std.StdDeserializer; -import org.codehaus.jackson.map.ext.JodaDeserializers; -import org.codehaus.jackson.map.module.SimpleModule; -import org.codehaus.jackson.map.ser.std.ToStringSerializer; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.deser.DurationDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.PeriodDeserializer; import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; +import org.joda.time.Duration; import org.joda.time.Interval; -import org.joda.time.ReadableInstant; +import org.joda.time.Period; import org.joda.time.format.ISODateTimeFormat; import java.io.IOException; @@ -47,6 +49,10 @@ public class JodaStuff module.addSerializer(DateTime.class, ToStringSerializer.instance); module.addDeserializer(Interval.class, new JodaStuff.IntervalDeserializer()); module.addSerializer(Interval.class, ToStringSerializer.instance); + module.addDeserializer(Period.class, new PeriodDeserializer()); + module.addSerializer(Period.class, ToStringSerializer.instance); + module.addDeserializer(Duration.class, new DurationDeserializer()); + module.addSerializer(Duration.class, ToStringSerializer.instance); return module; } diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index de0a297c569..ec95cb6b586 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -19,10 +19,11 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index c280507e207..f221b0e26df 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -19,10 +19,11 @@ package com.metamx.druid.histogram; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.Histogram; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java new file mode 100644 index 00000000000..aff4ad208c6 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import junit.framework.Assert; +import org.joda.time.DateTime; +import org.junit.Test; + +/** + */ +public class DefaultObjectMapperTest +{ + ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testDateTime() throws Exception + { + final DateTime time = new DateTime(); + + Assert.assertEquals(String.format("\"%s\"", time), mapper.writeValueAsString(time)); + } +} diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index f431915bbac..592a713cf3a 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java index 66c071a79fe..520b0188171 100644 --- a/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java +++ b/examples/rand/src/main/java/druid/examples/RandomFirehoseFactory.java @@ -1,5 +1,8 @@ package druid.examples; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.guava.Runnables; @@ -7,9 +10,9 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import java.io.IOException; import java.util.HashMap; diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index 4ffd7119431..ecdf2606a3b 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ package druid.examples; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -13,7 +14,7 @@ import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 9ab850948d9..a96af1c08a0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,5 +1,6 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -75,20 +76,20 @@ icu4j - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index fd072a427f3..e936d481489 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -1,5 +1,6 @@ package druid.examples; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -14,7 +15,7 @@ import com.metamx.druid.realtime.RealtimeNode; import com.metamx.druid.loading.SegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; -import org.codehaus.jackson.map.jsontype.NamedType; + import java.io.File; import java.io.IOException; diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 992cd239487..1bef29dfc6c 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -1,14 +1,17 @@ package druid.examples.twitter; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; + + + import twitter4j.ConnectionLifeCycleListener; import twitter4j.HashtagEntity; import twitter4j.Status; diff --git a/index-common/pom.xml b/index-common/pom.xml index 9d17c6b2c53..fee0b62da1f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -52,8 +53,8 @@ extendedset - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core net.java.dev.jets3t diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java index 2be3a50a595..69cb7ed628a 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.column; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java index 7f9b7ac7ee2..587c2455652 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.column; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.druid.index.serde.ColumnPartSerde; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java index d60428fbe3e..d9e53011429 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java @@ -19,9 +19,11 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.index.column.ColumnBuilder; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java index 760fcbb3f20..0dc17bdb0dd 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java @@ -19,12 +19,14 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.kv.GenericIndexed; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java index d30c6c6bbc0..5b542803c6c 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java @@ -19,6 +19,8 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.IAE; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; @@ -27,8 +29,8 @@ import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java index 87cb95758db..2d0d5a0dc09 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java index 0d140751e4c..8304062b05b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java +++ b/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java @@ -19,11 +19,13 @@ package com.metamx.druid.index.serde; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.ByteBuffer; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 34a8e06962d..6d3badd4c35 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -61,7 +62,7 @@ import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import java.io.ByteArrayOutputStream; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java index 4a90f35a288..066d6bf47ed 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.v1.serde; -import org.codehaus.jackson.map.ObjectMapper; + +import com.fasterxml.jackson.databind.ObjectMapper; /** * This is a "factory" interface for registering handlers in the system. It exists because I'm unaware of diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java index 05613567b87..cf2a2d56d0d 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.CSVParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java index 668dbc00fff..149a1946c42 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java index d372d46696f..177f0a2c358 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.common.parsers.DelimitedParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java index 11228bc132f..74255c2b261 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 3721f2c14f7..e04bd3d1d7c 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -27,8 +29,8 @@ import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.List; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java index 4905f268ed4..0f52bae8bd2 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java @@ -19,10 +19,12 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.metamx.common.parsers.ParserUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import java.util.Map; diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java index 2156314b4c6..2882d1a813a 100644 --- a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java +++ b/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java @@ -19,9 +19,10 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; -import org.codehaus.jackson.annotate.JsonValue; + import java.util.List; diff --git a/indexer/pom.xml b/indexer/pom.xml index d38a3a17769..2af06cfbb0c 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -79,12 +80,12 @@ - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind org.jdbi diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 720242466e4..a7dbee1f4e2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; @@ -30,7 +31,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 13490e3ef00..d5bb8a4925a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -63,7 +64,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java index b020a99fa4e..f9749b79861 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList; import com.metamx.common.Pair; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 3d682dadce0..eed2339114b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -19,6 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -54,11 +59,11 @@ import com.metamx.druid.utils.JodaUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.JsonGenerator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + + + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -97,24 +102,24 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromMap(Map argSpec) { - if (argSpec.containsKey("registererers")) { - List registererers = Lists.transform( - MapUtils.getList(argSpec, "registererers"), - new Function() + List registererers = Lists.transform( + MapUtils.getList(argSpec, "registererers", ImmutableList.of()), + new Function() + { + @Override + public Registererer apply(@Nullable Object input) { - @Override - public Registererer apply(@Nullable Object input) - { - try { - return (Registererer) Class.forName((String) input).newInstance(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + try { + return (Registererer) Class.forName((String) input).newInstance(); + } + catch (Exception e) { + throw Throwables.propagate(e); } } - ); + } + ); + if (!registererers.isEmpty()) { RegisteringNode.registerHandlers(registererers, Arrays.asList(jsonMapper)); } @@ -125,13 +130,7 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromFile(File file) { try { - return fromMap( - (Map) jsonMapper.readValue( - file, new TypeReference>() - { - } - ) - ); + return fromMap((Map) jsonMapper.readValue(file, new TypeReference>(){})); } catch (IOException e) { throw Throwables.propagate(e); @@ -191,14 +190,14 @@ public class HadoopDruidIndexerConfig public HadoopDruidIndexerConfig( final @JsonProperty("intervals") List intervals, final @JsonProperty("dataSource") String dataSource, - final @JsonProperty("timestampColumnName") String timestampColumnName, + final @JsonProperty("timestampColumn") String timestampColumnName, final @JsonProperty("timestampFormat") String timestampFormat, final @JsonProperty("dataSpec") DataSpec dataSpec, final @JsonProperty("segmentGranularity") Granularity segmentGranularity, final @JsonProperty("granularitySpec") GranularitySpec granularitySpec, final @JsonProperty("pathSpec") PathSpec pathSpec, - final @JsonProperty("jobOutputDir") String jobOutputDir, - final @JsonProperty("segmentOutputDir") String segmentOutputDir, + final @JsonProperty("workingPath") String jobOutputDir, + final @JsonProperty("segmentOutputPath") String segmentOutputDir, final @JsonProperty("version") DateTime version, final @JsonProperty("partitionDimension") String partitionDimension, final @JsonProperty("targetPartitionSize") Long targetPartitionSize, @@ -221,7 +220,7 @@ public class HadoopDruidIndexerConfig this.pathSpec = pathSpec; this.jobOutputDir = jobOutputDir; this.segmentOutputDir = segmentOutputDir; - this.version = version; + this.version = version == null ? new DateTime() : version; this.partitionsSpec = partitionsSpec; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java index 15354d0be67..62441560573 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java @@ -19,11 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import org.codehaus.jackson.map.jsontype.NamedType; + +import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.File; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java index d7f40087ba8..c0d692c7c5f 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * ShardSpec + a shard ID that is unique across this run. The shard ID is used for grouping and partitioning. diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java index aa1c3ed2f4b..97b046076e3 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/Utils.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.metamx.common.ISE; @@ -30,8 +32,8 @@ import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import java.io.IOException; import java.io.InputStream; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java index 151055271ec..e79af1e1df5 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java @@ -19,13 +19,15 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java index 30ce2df1171..ee73f4fdb25 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java @@ -19,9 +19,11 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Optional; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java index 51d2f37d437..22696a69c5d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java @@ -19,6 +19,8 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -26,8 +28,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java index 2d00cf71f06..7988fb49c5b 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java @@ -1,7 +1,11 @@ package com.metamx.druid.indexer.partitions; -import org.codehaus.jackson.annotate.JsonIgnore; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; @@ -14,6 +18,7 @@ public class PartitionsSpec private final boolean assumeGrouped; + @JsonCreator public PartitionsSpec( @JsonProperty("partitionDimension") @Nullable String partitionDimension, @JsonProperty("targetPartitionSize") @Nullable Long targetPartitionSize, diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java index 47e1676f848..3bcfb30c31d 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -32,7 +33,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonProperty; + import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java index aeca725bd1c..fb5b9e47ffe 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.guava.Comparators; @@ -30,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.DateTimeFormat; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java index 776f51c99bd..4a0b7278ad7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java @@ -19,16 +19,18 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import java.io.IOException; /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="type") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @JsonSubTypes(value={ @JsonSubTypes.Type(name="granular_unprocessed", value=GranularUnprocessedPathSpec.class), @JsonSubTypes.Type(name="granularity", value=GranularityPathSpec.class), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java index f46135c32cc..9a244c2f74a 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.path; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.codehaus.jackson.annotate.JsonProperty; + import java.io.IOException; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java index e2e1f9ce032..a223589db08 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java @@ -19,10 +19,11 @@ package com.metamx.druid.indexer.rollup; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.Granularity; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; -import org.codehaus.jackson.annotate.JsonProperty; + import java.util.List; diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java index 9bf40d86657..7a8892e9160 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java index 3523a198ba6..2957c1b3bf7 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java @@ -19,8 +19,9 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java index be1668afa93..1199e1ddf44 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java @@ -19,7 +19,8 @@ package com.metamx.druid.indexer.updater; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index f4db1148327..5fdff8ce8b8 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -39,8 +40,8 @@ public class HadoopDruidIndexerConfigTest try { cfg = jsonMapper.readValue( - "{" - + "\"granularitySpec\":{" + "{" + + " \"granularitySpec\":{" + " \"type\":\"uniform\"," + " \"gran\":\"hour\"," + " \"intervals\":[\"2012-01-01/P1D\"]" diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java index 0044d7d13e6..ecf6918b289 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java @@ -19,11 +19,12 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java index ab21be5f9f5..ea29d5ae727 100644 --- a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java +++ b/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java @@ -19,12 +19,13 @@ package com.metamx.druid.indexer.granularity; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import com.metamx.druid.jackson.DefaultObjectMapper; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/merger/pom.xml b/merger/pom.xml index 980aa96d5f0..7f91b1e8b92 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -113,16 +114,16 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind javax.inject diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java index d9479a7dbbf..9abb60d063a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java @@ -19,10 +19,12 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index 39b38a18a93..d6cb18093b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -19,13 +19,15 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.task.Task; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Collections; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 74a546cf696..0cebe1fc91c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.common; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.loading.S3SegmentPuller; import com.metamx.druid.loading.S3SegmentGetterConfig; @@ -28,7 +29,7 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.loading.SegmentPusher; import com.metamx.emitter.service.ServiceEmitter; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java index 28b78f78ad6..1ea40a95d2a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -31,10 +35,10 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.jets3t.service.S3Service; import org.jets3t.service.model.S3Object; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 634fe65ebaf..7d456d29e01 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -19,6 +19,10 @@ package com.metamx.druid.merger.common.index; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -39,10 +43,10 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.PlumberSchool; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + import org.joda.time.Interval; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 6de3eb0d73f..ae383f49f60 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -19,12 +19,14 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; public abstract class AbstractTask implements Task diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index f1153e5c43c..b3c90fa1330 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -33,8 +35,8 @@ import com.metamx.druid.index.v1.IndexableAdapter; import com.metamx.druid.index.v1.QueryableIndexIndexableAdapter; import com.metamx.druid.index.v1.Rowboat; import com.metamx.druid.index.v1.RowboatFilteringIndexAdapter; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index e17db4b980e..4ba3ab38030 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -28,8 +30,8 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; import java.io.File; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f08298cd2d..5f37ad2853e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; @@ -33,9 +35,9 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.map.ObjectMapper; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 55edfdbc3bc..110fdbf86f6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -38,8 +40,8 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.Interval; import javax.annotation.Nullable; @@ -49,7 +51,8 @@ import java.util.Set; public class IndexDeterminePartitionsTask extends AbstractTask { - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final FirehoseFactory firehoseFactory; @JsonProperty private final Schema schema; @JsonProperty private final long targetPartitionSize; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b89142ef19a..b3da8978bd3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -36,8 +38,8 @@ import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 6074765ddbf..ca514cec52b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; @@ -31,8 +33,8 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 20fefa0014f..21f6c1e6416 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,6 +19,9 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -43,9 +46,9 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 807172d11ae..60d1cf21e39 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -19,14 +19,16 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask; import com.metamx.druid.merger.common.task.IndexGeneratorTask; import com.metamx.druid.merger.common.task.IndexTask; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java index 1a718fdaf20..32e8bfd71dd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -1,9 +1,10 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index c1dbce4fc61..41032cc831f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -29,7 +30,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index bc53ef0d4f7..f226a19be4f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; @@ -27,7 +28,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.DBI; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a95f64cb623..4157d6a66ca 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; @@ -48,7 +49,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import com.netflix.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java index fcbb2450cbc..d3a32aca462 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Set; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index c6353698cbd..8203781bbf9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -27,7 +28,7 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java index 26f0105771d..5b342883bb2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.config; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; -import org.codehaus.jackson.annotate.JsonProperty; + import org.skife.config.Config; public abstract class IndexerDbConnectorConfig extends DbConnectorConfig diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3dae4046764..7d075cbe9e8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator.http; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.services.ec2.AmazonEC2Client; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -89,8 +91,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 4cc1df9fa6f..1012beb830b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.coordinator.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; @@ -26,8 +28,8 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 8d51da61afd..16b629d9abb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -28,6 +28,7 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; @@ -36,7 +37,7 @@ import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.map.ObjectMapper; + import javax.annotation.Nullable; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java index 8d302df25f6..4c2b86f4f6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java index 876a2635273..76061637312 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java @@ -19,8 +19,9 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index 8395fa2d6c8..fada73cb40e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -19,8 +19,11 @@ package com.metamx.druid.merger.coordinator.setup; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java index 5e43e68ae66..89a0dd2d5c1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator.setup; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.ISE; @@ -28,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import org.apache.commons.collections.MapUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.FoldController; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 5fc49788fcd..c41f4ac8be4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -32,7 +32,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.commons.io.FileUtils; -import org.codehaus.jackson.map.ObjectMapper; + import java.io.File; import java.util.concurrent.ExecutorService; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java index a1ebf273521..6b349de38f7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.merger.worker.config.WorkerConfig; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + /** * A container for worker metadata. diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index d4237da7a9e..ab3a35d53de 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.worker; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -32,7 +33,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.state.ConnectionState; import com.netflix.curator.framework.state.ConnectionStateListener; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.util.Arrays; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 0799a8de37c..237ae09d854 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.worker.http; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -59,8 +61,8 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index b7107ed72d2..ded6b55d64d 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,5 +1,10 @@ package com.metamx.druid.merger.coordinator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -29,10 +34,10 @@ import com.netflix.curator.retry.ExponentialBackoffRetry; import com.netflix.curator.test.TestingCluster; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.zookeeper.CreateMode; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonTypeName; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.map.jsontype.NamedType; + + + + import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -495,6 +500,7 @@ public class RemoteTaskRunnerTest private final List segments; private final List aggregators; + @JsonCreator public TestTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, diff --git a/pom.xml b/pom.xml index aa77ea6894b..7b1e3f9e062 100644 --- a/pom.xml +++ b/pom.xml @@ -38,6 +38,7 @@ UTF-8 + 0.20.0 @@ -58,17 +59,17 @@ com.metamx emitter - 0.0.7 + 0.2.0 com.metamx http-client - 0.6.1 + 0.7.0 com.metamx java-util - 0.19.1 + ${metamx.java-util.version} com.metamx @@ -162,24 +163,34 @@ 4.8.1 - org.codehaus.jackson - jackson-core-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-annotations + 2.1.2 - org.codehaus.jackson - jackson-jaxrs - 1.9.9 + com.fasterxml.jackson.core + jackson-core + 2.1.3 - org.codehaus.jackson - jackson-mapper-asl - 1.9.9 + com.fasterxml.jackson.core + jackson-databind + 2.1.3 - org.codehaus.jackson - jackson-smile - 1.9.9 + com.fasterxml.jackson.datatype + jackson-datatype-joda + 2.1.2 + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + 2.1.3 + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + 2.1.3 javax.inject @@ -269,7 +280,7 @@ java-util test-jar test - 0.16.0 + ${metamx.java-util.version} diff --git a/realtime/pom.xml b/realtime/pom.xml index a8c50128f2b..26c23ad9133 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -82,16 +83,16 @@ guava - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile org.jdbi diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java index b5709de1d6b..aab4509bbe5 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index b2afbbc4884..d98997b5051 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.codehaus.jackson.annotate.JsonProperty; + import org.joda.time.Period; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java index e090e088d08..40d0e662b49 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java @@ -19,8 +19,11 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.io.IOException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java index 05714ddfdbd..12c74ad6b16 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/KafkaFirehoseFactory.java @@ -19,6 +19,8 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.metamx.common.exception.FormattedException; @@ -30,8 +32,8 @@ import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaMessageStream; import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.Message; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.io.IOException; import java.nio.CharBuffer; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java index 5bb10b36753..2d377124cc3 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/MetadataUpdater.java @@ -19,13 +19,14 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.phonebook.PhoneBook; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java index 0f0d87b3490..5fcc1f29f7d 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java @@ -19,8 +19,9 @@ package com.metamx.druid.realtime; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; /** */ diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index f503e80ade3..d8fc7ebeea7 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -19,6 +19,12 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -47,12 +53,12 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.BeanProperty; -import org.codehaus.jackson.map.DeserializationContext; -import org.codehaus.jackson.map.InjectableValues; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; -import org.codehaus.jackson.type.TypeReference; + + + + + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index f4df5e054f8..47a4ef0f40f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -19,6 +19,11 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -53,11 +58,11 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.io.FileUtils; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; -import org.codehaus.jackson.map.annotate.JacksonInject; + + + + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 1b22f1e3a78..5a74b17e223 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -19,9 +19,10 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.SegmentPusher; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** diff --git a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java index b54c8efbbf6..ee3d63a14f8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java @@ -19,14 +19,16 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import java.util.Arrays; diff --git a/server/pom.xml b/server/pom.xml index 544108cef5f..66309b73c30 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server @@ -107,20 +108,20 @@ guice-servlet - org.codehaus.jackson - jackson-core-asl + com.fasterxml.jackson.core + jackson-core - org.codehaus.jackson - jackson-jaxrs + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider - org.codehaus.jackson - jackson-mapper-asl + com.fasterxml.jackson.core + jackson-databind - org.codehaus.jackson - jackson-smile + com.fasterxml.jackson.dataformat + jackson-dataformat-smile javax.inject @@ -154,10 +155,10 @@ joda-time joda-time - - com.google.code.findbugs - jsr305 - + + com.google.code.findbugs + jsr305 + log4j log4j diff --git a/server/src/main/java/com/metamx/TsvToJson.java b/server/src/main/java/com/metamx/TsvToJson.java index 9d4417fe597..65effd5077e 100644 --- a/server/src/main/java/com/metamx/TsvToJson.java +++ b/server/src/main/java/com/metamx/TsvToJson.java @@ -19,11 +19,12 @@ package com.metamx; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.collect.Maps; import com.metamx.common.IAE; import com.metamx.common.ISE; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/server/src/main/java/com/metamx/druid/BaseServerNode.java b/server/src/main/java/com/metamx/druid/BaseServerNode.java index cdc7a68e4f6..dbbe286e270 100644 --- a/server/src/main/java/com/metamx/druid/BaseServerNode.java +++ b/server/src/main/java/com/metamx/druid/BaseServerNode.java @@ -19,6 +19,7 @@ package com.metamx.druid; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -29,7 +30,7 @@ import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import org.codehaus.jackson.map.ObjectMapper; + import org.skife.config.ConfigurationObjectFactory; import java.nio.ByteBuffer; diff --git a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java index 4e68ad2995e..a6d342d8419 100644 --- a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + /** */ -@JsonTypeInfo(use=JsonTypeInfo.Id.NAME, property="action") +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="action") @JsonSubTypes(value={ @JsonSubTypes.Type(name="load", value=SegmentChangeRequestLoad.class), @JsonSubTypes.Type(name="drop", value=SegmentChangeRequestDrop.class) diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java index 0093907d26e..b231781e394 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java index c98106efdfb..ecf35513af9 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java @@ -19,10 +19,13 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; -import org.codehaus.jackson.annotate.JsonUnwrapped; + + + /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 1fb756daafe..57d64e0ba32 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -33,7 +34,7 @@ import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 2840eb037a8..e76b372978f 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -19,6 +19,8 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -29,8 +31,8 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index ef620cd1e38..b17a2f63299 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -19,6 +19,7 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -31,7 +32,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index c6d284403c0..2230932d9a1 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -46,8 +48,8 @@ import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.Monitor; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.smile.SmileFactory; + + import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 51df502fa64..5ba559d4656 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,6 +19,7 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -67,7 +68,7 @@ import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.x.discovery.ServiceDiscovery; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.ZkClient; -import org.codehaus.jackson.map.ObjectMapper; + import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 1d33550635c..47395f73eeb 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -19,6 +19,8 @@ package com.metamx.druid.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; @@ -27,8 +29,8 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import org.codehaus.jackson.jaxrs.JacksonJsonProvider; -import org.codehaus.jackson.map.ObjectMapper; + + import javax.inject.Singleton; diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java index 3fdcd5dbdc5..a609329b0fd 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java index b35c3ce4d7c..d54a78b8bba 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java +++ b/server/src/main/java/com/metamx/druid/http/SegmentToMove.java @@ -19,8 +19,9 @@ package com.metamx.druid.http; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java index f97d6dde379..2bf090850b6 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java @@ -19,10 +19,12 @@ package com.metamx.druid.index.brita; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; import com.metamx.druid.query.search.SearchQuerySpec; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java index 3069984739b..4628028ee0b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java @@ -19,8 +19,9 @@ package com.metamx.druid.index.v1; +import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.druid.QueryGranularity; -import org.codehaus.jackson.annotate.JsonCreator; + import org.joda.time.DateTime; import org.joda.time.Days; import org.joda.time.Hours; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java index 4d64c3ed464..5af4b905719 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java @@ -19,6 +19,7 @@ package com.metamx.druid.loading; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.Closeables; @@ -29,7 +30,7 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.codehaus.jackson.map.ObjectMapper; + import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 3d9c8ecffe0..281c4cf5c3d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; @@ -49,7 +50,7 @@ import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.Duration; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index e7f3d5f41ed..f3d93d0e6fd 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,6 +19,7 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -41,7 +42,7 @@ import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.ToStringResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java index 32c77fdff3d..07a0f8dba6e 100644 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java @@ -19,13 +19,14 @@ package com.metamx.druid.master; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merge.ClientAppendQuery; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.HttpResponseHandler; import com.netflix.curator.x.discovery.ServiceProvider; -import org.codehaus.jackson.map.ObjectMapper; + import java.net.URL; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java index 0acdd8bc2f8..1e9bef477b5 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java index 5aa984ccba8..bb50ef352fa 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index 152f074dc3a..22f1794189d 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -19,9 +19,11 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java index 051967e65ab..a332c302dd7 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java @@ -19,10 +19,12 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import org.codehaus.jackson.annotate.JsonCreator; -import org.codehaus.jackson.annotate.JsonProperty; + + import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index a6eced93c68..3d2ea2f59c1 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -19,12 +19,14 @@ package com.metamx.druid.master.rules; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; -import org.codehaus.jackson.annotate.JsonSubTypes; -import org.codehaus.jackson.annotate.JsonTypeInfo; + + import org.joda.time.DateTime; /** diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java index 62cb939e0ed..38a63d57f0c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java index 122b779e922..41ed201242c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java @@ -19,13 +19,15 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; + + import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index de148a9219c..762662741a6 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; @@ -60,7 +61,7 @@ import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.codehaus.jackson.type.TypeReference; + import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 5165647b24f..99019ef5568 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.coordination; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; @@ -33,7 +34,7 @@ import com.metamx.druid.loading.NoopSegmentLoader; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; import com.metamx.druid.shard.NoneShardSpec; -import org.codehaus.jackson.map.ObjectMapper; + import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; From ee193f0ca7e8b9f7676b7388114470c5beb67135 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 09:50:30 -0800 Subject: [PATCH 10/14] DeterminePartitionsJob: Select partition dimension to minimize segment size variance when cardinality is low --- .../druid/indexer/DeterminePartitionsJob.java | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d5bb8a4925a..4b4a5c8b995 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -500,6 +500,7 @@ public class DeterminePartitionsJob implements Jobby { private static final double SHARD_COMBINE_THRESHOLD = 0.25; private static final double SHARD_OVERSIZE_THRESHOLD = 1.5; + private static final int HIGH_CARDINALITY_THRESHOLD = 3000000; @Override protected void innerReduce( @@ -634,7 +635,9 @@ public class DeterminePartitionsJob implements Jobby final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = -1; + int maxCardinality = Integer.MIN_VALUE; + long minVariance = Long.MAX_VALUE; + DimPartitions minVariancePartitions = null; DimPartitions maxCardinalityPartitions = null; for(final DimPartitions dimPartitions : dimPartitionss.values()) { @@ -660,10 +663,18 @@ public class DeterminePartitionsJob implements Jobby continue; } - if(dimPartitions.getCardinality() > maxCardinality) { - maxCardinality = dimPartitions.getCardinality(); + final int cardinality = dimPartitions.getCardinality(); + final long variance = dimPartitions.getVariance(); + + if(cardinality > maxCardinality) { + maxCardinality = cardinality; maxCardinalityPartitions = dimPartitions; } + + if(variance < minVariance) { + minVariance = variance; + minVariancePartitions = dimPartitions; + } } if(maxCardinalityPartitions == null) { @@ -675,8 +686,12 @@ public class DeterminePartitionsJob implements Jobby context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); + final DimPartitions chosenPartitions = maxCardinality > HIGH_CARDINALITY_THRESHOLD + ? maxCardinalityPartitions + : minVariancePartitions; + final List chosenShardSpecs = Lists.transform( - maxCardinalityPartitions.partitions, new Function() + chosenPartitions.partitions, new Function() { @Override public ShardSpec apply(DimPartition dimPartition) @@ -752,6 +767,19 @@ public class DeterminePartitionsJob implements Jobby return sum; } + public long getVariance() + { + final long meanRows = getRows() / partitions.size(); + + long variance = 0; + for(final DimPartition dimPartition : partitions) { + variance += (dimPartition.rows - meanRows) * (dimPartition.rows - meanRows); + } + + variance /= partitions.size(); + return variance; + } + public int getRows() { int sum = 0; From e3822f6ca795cb7fc8385b97147959d8b16ce582 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 13 Feb 2013 10:16:03 -0800 Subject: [PATCH 11/14] DeterminePartitionsJob: Fix docs --- .../com/metamx/druid/indexer/DeterminePartitionsJob.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index 4b4a5c8b995..d4ee1941396 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -79,13 +79,16 @@ import java.util.Set; /** * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, - * choosing the highest cardinality dimension that satisfies the criteria: + * choosing the best dimension that satisfies the criteria: * *
    *
  • Must have exactly one value per row.
  • *
  • Must not generate oversized partitions. A dimension with N rows having the same value will necessarily * put all those rows in the same partition, and that partition may be much larger than the target size.
  • *
+ * + * "Best" means a very high cardinality dimension, or, if none exist, the dimension that minimizes segment size + * variance. */ public class DeterminePartitionsJob implements Jobby { From 1b85eaad3e39aea91461b0406a6a70b112ef09dd Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 12/14] simplify query toolchest interface --- .../metamx/druid/query/QueryToolChest.java | 12 ++---- .../TimeBoundaryQueryQueryToolChest.java | 39 +++++-------------- .../java/com/metamx/druid/LogicalSegment.java | 8 ++++ .../metamx/druid/TimelineObjectHolder.java | 3 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- 6 files changed, 25 insertions(+), 41 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/LogicalSegment.java diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index f729016032e..e2d227729d3 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -22,12 +22,9 @@ package com.metamx.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.emitter.service.ServiceMetricEvent; - - import java.util.List; /** @@ -50,7 +47,7 @@ public abstract class QueryToolChest makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); public abstract TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query) { + public CacheStrategy getCacheStrategy(QueryType query) { return null; } @@ -62,10 +59,7 @@ public abstract class QueryToolChest> filterSegments( - QueryType query, - List> segments - ) { + public List filterSegments(QueryType query, List segments) { return segments; } } diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 86e618bcee8..4ff1ffdf533 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,16 +22,14 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.Query; -import com.metamx.druid.TimelineObjectHolder; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; import com.metamx.druid.query.CacheStrategy; @@ -47,8 +45,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; -import java.util.Map; -import java.util.Set; /** */ @@ -64,32 +60,17 @@ public class TimeBoundaryQueryQueryToolChest { }; - @Override - public List> filterSegments( - TimeBoundaryQuery query, - List> input - ) { - long minMillis = Long.MAX_VALUE; - long maxMillis = Long.MIN_VALUE; - TimelineObjectHolder min = null; - TimelineObjectHolder max = null; - - for(TimelineObjectHolder e : input) { - final long start = e.getInterval().getStartMillis(); - final long end = e.getInterval().getEndMillis(); - - if(min == null || start < minMillis) { - min = e; - minMillis = start; - } - if(max == null || end > maxMillis) { - max = e; - maxMillis = end; - } + public List filterSegments(TimeBoundaryQuery query, List input) + { + if(input.size() <= 1) { + return input; } - return min == max ? Lists.newArrayList(min) : Lists.newArrayList(min , max); + return Lists.newArrayList( + Iterables.getFirst(input, null), + Iterables.getLast(input) + ); } @Override diff --git a/common/src/main/java/com/metamx/druid/LogicalSegment.java b/common/src/main/java/com/metamx/druid/LogicalSegment.java new file mode 100644 index 00000000000..5550fb44966 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/LogicalSegment.java @@ -0,0 +1,8 @@ +package com.metamx.druid; + +import org.joda.time.Interval; + +public interface LogicalSegment +{ + public Interval getInterval(); +} diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index 403fe8bb2cb..d93e5ed8f6b 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -24,7 +24,7 @@ import org.joda.time.Interval; /** */ -public class TimelineObjectHolder +public class TimelineObjectHolder implements LogicalSegment { private final Interval interval; private final VersionType version; @@ -41,6 +41,7 @@ public class TimelineObjectHolder this.object = object; } + @Override public Interval getInterval() { return interval; diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 592a713cf3a..c496170be99 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a96af1c08a0..a327c94881d 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -10,7 +10,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.3.0-SNAPSHOT From 6998d604a293f752af8649c7f12b31f2801decaa Mon Sep 17 00:00:00 2001 From: xvrl Date: Thu, 14 Feb 2013 12:41:34 -0800 Subject: [PATCH 13/14] fixes #80 --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 4ff1ffdf533..9701d9eee76 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -41,7 +41,6 @@ import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.DateTime; - import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; From 780410446c019f748e23d586496c3267932c8126 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 15 Feb 2013 11:19:13 -0800 Subject: [PATCH 14/14] even more simple --- .../query/timeboundary/TimeBoundaryQueryQueryToolChest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9701d9eee76..e8735522f89 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; @@ -67,8 +66,8 @@ public class TimeBoundaryQueryQueryToolChest } return Lists.newArrayList( - Iterables.getFirst(input, null), - Iterables.getLast(input) + input.get(0), + input.get(input.size() - 1) ); }