From e10025c84172ba448b6ac834e5fe784eb6fe5c84 Mon Sep 17 00:00:00 2001 From: xvrl Date: Fri, 8 Feb 2013 16:50:23 -0800 Subject: [PATCH 01/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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/42] 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 ee4236dac2e5cbec65ef65f9c6ba365637ee9561 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 17:10:27 -0600 Subject: [PATCH 14/42] 1) Update to mmx build of jackson 2.1 branch to get bug-fix for jackson-databind #167 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1f1dd515ff..dc4e48422c8 100644 --- a/pom.xml +++ b/pom.xml @@ -175,7 +175,7 @@ com.fasterxml.jackson.core jackson-databind - 2.1.3 + 2.1.4-mmx-2 com.fasterxml.jackson.datatype From 4f11eb5209a98e909fc3aa0b6b08cc9ed6463c32 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 18:24:51 -0600 Subject: [PATCH 15/42] 1) Fix alert sent from line DruidMasterBalancer:71 to not include all of the segments that a server is holding --- .../main/java/com/metamx/druid/master/DruidMasterBalancer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index 4ad3a839b4b..c3853af91f0 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -70,7 +70,7 @@ public class DruidMasterBalancer implements DruidMasterHelper if (holder.getLifetime() <= 0) { log.makeAlert("[%s]: Balancer move segments queue has a segment stuck", tier) .addData("segment", holder.getSegment().getIdentifier()) - .addData("server", holder.getServer()) + .addData("server", holder.getServer().getStringProps()) .emit(); } } From e56bc27f93fc355782713a51b3eafe2af3d0bef8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 16:27:12 -0800 Subject: [PATCH 16/42] [maven-release-plugin] prepare release druid-0.2.8 --- client/pom.xml | 5 ++--- common/pom.xml | 5 ++--- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 5 ++--- examples/twitter/pom.xml | 5 ++--- index-common/pom.xml | 5 ++--- indexer/pom.xml | 5 ++--- merger/pom.xml | 5 ++--- pom.xml | 2 +- realtime/pom.xml | 5 ++--- server/pom.xml | 5 ++--- 12 files changed, 22 insertions(+), 31 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 588da3fe4de..117b634c7a8 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/common/pom.xml b/common/pom.xml index 369aa402c4d..7c29586abb0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-common @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 9b2567dc753..4aa4a010a02 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.2.8 com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/pom.xml b/examples/pom.xml index fa195a3f0f4..356773392f2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 592a713cf3a..21a6f40303c 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a96af1c08a0..7f3ee6e940c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/index-common/pom.xml b/index-common/pom.xml index 084bc314ba1..77641cc61c2 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/indexer/pom.xml b/indexer/pom.xml index 5ff19d88607..75bf11e4f0b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/merger/pom.xml b/merger/pom.xml index da776d6e512..26d410f3059 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/pom.xml b/pom.xml index dc4e48422c8..95214118926 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8-SNAPSHOT + 0.2.8 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c321afa695d..8e7744e01f7 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 diff --git a/server/pom.xml b/server/pom.xml index 9b4a0707961..2478372e2dd 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server @@ -29,7 +28,7 @@ com.metamx druid - 0.2.8-SNAPSHOT + 0.2.8 From 152bf201ec35b56b3334e058330ed763adbbf260 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 16:27:20 -0800 Subject: [PATCH 17/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 117b634c7a8..8329d3bb8a4 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 7c29586abb0..978164b92e5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4aa4a010a02..f52cc4d954b 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.8 + 0.2.9-SNAPSHOT com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 356773392f2..aa8cb80def3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 21a6f40303c..978f04351df 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7f3ee6e940c..924c439aa09 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 77641cc61c2..4b19856be9c 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 75bf11e4f0b..bb4a90b692f 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 26d410f3059..93b979a1460 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/pom.xml b/pom.xml index 95214118926..e0e1f947cd2 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.8 + 0.2.9-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8e7744e01f7..dd267cee537 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2478372e2dd..4210d090b5a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.8 + 0.2.9-SNAPSHOT From 48538d045409f3d996bc06fe3a191ca7ccfb05af Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 14 Feb 2013 22:59:17 -0600 Subject: [PATCH 18/42] 1) Fix bug in index converter when column has cardinality 0 --- .../com/metamx/druid/index/v1/IndexIO.java | 59 +++++++++++-------- 1 file changed, 34 insertions(+), 25 deletions(-) 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 6d3badd4c35..5a2c7307ddd 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 @@ -370,6 +370,7 @@ public class IndexIO } LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); + Set skippedDimensions = Sets.newLinkedHashSet(); for (String filename : v8SmooshedFiles.getInternalFilenames()) { log.info("Processing file[%s]", filename); if (filename.startsWith("dim_")) { @@ -392,6 +393,12 @@ public class IndexIO dimBuffer, GenericIndexed.stringStrategy ); + if (dictionary.size() == 0) { + log.info("Dimension[%s] had cardinality 0, equivalent to no column, so skipping.", dimension); + skippedDimensions.add(dimension); + continue; + } + VSizeIndexedInts singleValCol = null; VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer()); GenericIndexed bitmaps = bitmapIndexes.get(dimension); @@ -555,35 +562,37 @@ public class IndexIO channel.write(ByteBuffer.wrap(specBytes)); serdeficator.write(channel); channel.close(); - } else if ("index.drd".equals(filename)) { - final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile(filename); - - indexBuffer.get(); // Skip the version byte - final GenericIndexed dims = GenericIndexed.read( - indexBuffer, GenericIndexed.stringStrategy - ); - final GenericIndexed availableMetrics = GenericIndexed.read( - indexBuffer, GenericIndexed.stringStrategy - ); - final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); - - Set columns = Sets.newTreeSet(); - columns.addAll(Lists.newArrayList(dims)); - columns.addAll(Lists.newArrayList(availableMetrics)); - - GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); - - final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; - final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); - cols.writeToChannel(writer); - dims.writeToChannel(writer); - serializerUtils.writeLong(writer, dataInterval.getStartMillis()); - serializerUtils.writeLong(writer, dataInterval.getEndMillis()); - writer.close(); } else { skippedFiles.add(filename); } } + + final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile("index.drd"); + + indexBuffer.get(); // Skip the version byte + final GenericIndexed dims = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final GenericIndexed availableMetrics = GenericIndexed.read( + indexBuffer, GenericIndexed.stringStrategy + ); + final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); + + Set columns = Sets.newTreeSet(); + columns.addAll(Lists.newArrayList(dims)); + columns.addAll(Lists.newArrayList(availableMetrics)); + columns.removeAll(skippedDimensions); + + GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); + + final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); + cols.writeToChannel(writer); + dims.writeToChannel(writer); + serializerUtils.writeLong(writer, dataInterval.getStartMillis()); + serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + writer.close(); + log.info("Skipped files[%s]", skippedFiles); v9Smoosher.close(); From 22d1f2a0c09bed8b77983ed9aa271fbd05070857 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:07:17 -0800 Subject: [PATCH 19/42] IndexMergerTest: Empty column test --- .../druid/index/v1/IndexMergerTest.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 097762106a2..86d03f6dd76 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.index.v1; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.Files; @@ -111,4 +112,33 @@ public class IndexMergerTest FileUtils.deleteQuietly(mergedDir); } } + + @Test + public void testPersistEmptyColumn() throws Exception + { + final IncrementalIndex toPersist = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final File tmpDir = Files.createTempDir(); + + try { + toPersist.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "foo") + ) + ); + + final QueryableIndex merged = IndexIO.loadIndex( + IndexMerger.persist(toPersist, tmpDir) + ); + + Assert.assertEquals(1, merged.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim1", "dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); + Assert.assertEquals(null, merged.getColumn("dim1")); + } finally { + FileUtils.deleteQuietly(tmpDir); + } + + + } } From ba7df5b7092dbc85a687c3036091d872e8eb1dce Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:23:41 -0800 Subject: [PATCH 20/42] [maven-release-plugin] prepare release druid-0.2.9 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 8329d3bb8a4..a5689080ff3 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/common/pom.xml b/common/pom.xml index 978164b92e5..f906565c5fb 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index f52cc4d954b..08d00318d79 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.9-SNAPSHOT + 0.2.9 com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/pom.xml b/examples/pom.xml index aa8cb80def3..0c9853282a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 978f04351df..5b97b7088d9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 924c439aa09..771c0e72e5c 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/index-common/pom.xml b/index-common/pom.xml index 4b19856be9c..f6879ba8336 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/indexer/pom.xml b/indexer/pom.xml index bb4a90b692f..485cde8a6bb 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/merger/pom.xml b/merger/pom.xml index 93b979a1460..8378522c36c 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/pom.xml b/pom.xml index e0e1f947cd2..a2fb097fd71 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.9-SNAPSHOT + 0.2.9 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index dd267cee537..042bcfc26f0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 diff --git a/server/pom.xml b/server/pom.xml index 4210d090b5a..af0009bbd9d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9-SNAPSHOT + 0.2.9 From 3fed8ee0cee1c9e164731e067b1f800f2afcf7d4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 21:23:46 -0800 Subject: [PATCH 21/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index a5689080ff3..302778496bc 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f906565c5fb..f2f878a6da1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 08d00318d79..90716448ca1 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.9 + 0.2.10-SNAPSHOT com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 0c9853282a7..bfbdeb2f735 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 5b97b7088d9..20bf45f8cae 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 771c0e72e5c..42be38a8916 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f6879ba8336..49db8c7b958 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 485cde8a6bb..ee05ae99737 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 8378522c36c..cc230b8e921 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/pom.xml b/pom.xml index a2fb097fd71..3166c2e845b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.9 + 0.2.10-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 042bcfc26f0..0c3e8ab773c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index af0009bbd9d..8e715da4e8e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.9 + 0.2.10-SNAPSHOT From 548c901c0b21a21fe5f93c115a8750e00b39cebf Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 14 Feb 2013 23:34:49 -0800 Subject: [PATCH 22/42] Additional fix for columns with cardinality 0 --- .../com/metamx/druid/index/v1/IndexIO.java | 28 +++++++++---- .../druid/index/v1/IndexMergerTest.java | 39 ++++++++++++++----- 2 files changed, 49 insertions(+), 18 deletions(-) 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 5a2c7307ddd..d26e73f5b3c 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 @@ -21,6 +21,7 @@ package com.metamx.druid.index.v1; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -62,7 +63,6 @@ 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.joda.time.Interval; import java.io.ByteArrayOutputStream; @@ -369,8 +369,8 @@ public class IndexIO ); } - LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); - Set skippedDimensions = Sets.newLinkedHashSet(); + final LinkedHashSet skippedFiles = Sets.newLinkedHashSet(); + final Set skippedDimensions = Sets.newLinkedHashSet(); for (String filename : v8SmooshedFiles.getInternalFilenames()) { log.info("Processing file[%s]", filename); if (filename.startsWith("dim_")) { @@ -570,25 +570,37 @@ public class IndexIO final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile("index.drd"); indexBuffer.get(); // Skip the version byte - final GenericIndexed dims = GenericIndexed.read( + final GenericIndexed dims8 = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); + final GenericIndexed dims9 = GenericIndexed.fromIterable( + Iterables.filter( + dims8, new Predicate() + { + @Override + public boolean apply(String s) + { + return !skippedDimensions.contains(s); + } + } + ), + GenericIndexed.stringStrategy + ); final GenericIndexed availableMetrics = GenericIndexed.read( indexBuffer, GenericIndexed.stringStrategy ); final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer)); Set columns = Sets.newTreeSet(); - columns.addAll(Lists.newArrayList(dims)); + columns.addAll(Lists.newArrayList(dims9)); columns.addAll(Lists.newArrayList(availableMetrics)); - columns.removeAll(skippedDimensions); GenericIndexed cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy); - final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16; + final int numBytes = cols.getSerializedSize() + dims9.getSerializedSize() + 16; final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); cols.writeToChannel(writer); - dims.writeToChannel(writer); + dims9.writeToChannel(writer); serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis()); writer.close(); diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 86d03f6dd76..407cd36253f 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -33,6 +33,7 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; +import java.util.List; /** */ @@ -116,11 +117,14 @@ public class IndexMergerTest @Test public void testPersistEmptyColumn() throws Exception { - final IncrementalIndex toPersist = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); - final File tmpDir = Files.createTempDir(); + final IncrementalIndex toPersist1 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final IncrementalIndex toPersist2 = new IncrementalIndex(0L, QueryGranularity.NONE, new AggregatorFactory[]{}); + final File tmpDir1 = Files.createTempDir(); + final File tmpDir2 = Files.createTempDir(); + final File tmpDir3 = Files.createTempDir(); try { - toPersist.add( + toPersist1.add( new MapBasedInputRow( 1L, ImmutableList.of("dim1", "dim2"), @@ -128,17 +132,32 @@ public class IndexMergerTest ) ); - final QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.persist(toPersist, tmpDir) + toPersist2.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "bar") + ) ); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); + final QueryableIndex merged = IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) + ); + + Assert.assertEquals(1, index1.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index1.getAvailableDimensions())); + + Assert.assertEquals(1, index2.getTimeColumn().getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index2.getAvailableDimensions())); + Assert.assertEquals(1, merged.getTimeColumn().getLength()); - Assert.assertEquals(ImmutableList.of("dim1", "dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); - Assert.assertEquals(null, merged.getColumn("dim1")); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); } finally { - FileUtils.deleteQuietly(tmpDir); + FileUtils.deleteQuietly(tmpDir1); + FileUtils.deleteQuietly(tmpDir2); + FileUtils.deleteQuietly(tmpDir3); } - - } } From 5ee7c0c264c99c4d704231429455867c16094117 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 15 Feb 2013 12:15:53 -0600 Subject: [PATCH 23/42] 1) Remove stray import of java.util.List --- .../src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 407cd36253f..9cfe9d22379 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -33,7 +33,6 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; -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 24/42] 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) ); } From 0d99cee3c3fff7802009d4a3609acd10a07102e7 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Fri, 15 Feb 2013 14:25:44 -0600 Subject: [PATCH 25/42] 1) Whitespace ftw! --- .../timeboundary/TimeBoundaryQueryQueryToolChest.java | 7 ++----- 1 file changed, 2 insertions(+), 5 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 e8735522f89..dcf09526a31 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 @@ -38,8 +38,8 @@ 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.joda.time.DateTime; + import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; @@ -65,10 +65,7 @@ public class TimeBoundaryQueryQueryToolChest return input; } - return Lists.newArrayList( - input.get(0), - input.get(input.size() - 1) - ); + return Lists.newArrayList(input.get(0), input.get(input.size() - 1)); } @Override From 6bbc992101d37b5ddd225d5bd05b5cf763b821a0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:32 -0800 Subject: [PATCH 26/42] [maven-release-plugin] prepare release druid-0.3.0 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 5 ++--- examples/twitter/pom.xml | 5 ++--- index-common/pom.xml | 5 ++--- indexer/pom.xml | 5 ++--- merger/pom.xml | 5 ++--- pom.xml | 2 +- realtime/pom.xml | 5 ++--- server/pom.xml | 2 +- 12 files changed, 19 insertions(+), 25 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 602d4b9cbb7..c82151f4498 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/common/pom.xml b/common/pom.xml index 0cb9bf4a429..78562c476cd 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 88994a529d5..e9ccc47ec39 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0-SNAPSHOT + 0.3.0 com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/pom.xml b/examples/pom.xml index b24a0f7dd42..dc880f384e2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index c496170be99..afb3e13decb 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-rand @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a327c94881d..7e04b88abf1 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -1,6 +1,5 @@ - + 4.0.0 com.metamx.druid druid-examples-twitter @@ -10,7 +9,7 @@ com.metamx druid-examples - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/index-common/pom.xml b/index-common/pom.xml index fee0b62da1f..67c71003775 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-index-common @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/indexer/pom.xml b/indexer/pom.xml index 2af06cfbb0c..e587f76f5d3 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-indexer @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/merger/pom.xml b/merger/pom.xml index 7f91b1e8b92..1d007233120 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-merger @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/pom.xml b/pom.xml index 0e2e7f1b0c0..85ddb331552 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0-SNAPSHOT + 0.3.0 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 26c23ad9133..215d3bf0cbe 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-realtime @@ -29,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 diff --git a/server/pom.xml b/server/pom.xml index 284c76ff474..049318700ef 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0-SNAPSHOT + 0.3.0 From 824e3c0eb2956b619028df4c2f8ff287f170fc1c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 15 Feb 2013 13:03:39 -0800 Subject: [PATCH 27/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index c82151f4498..59cc716195c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 78562c476cd..710c88d7274 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index e9ccc47ec39..01078c607f4 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.0 + 0.3.1-SNAPSHOT com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index dc880f384e2..4220ed243bc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index afb3e13decb..ae5e1e767e9 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 7e04b88abf1..c557c46800b 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 67c71003775..c67359cb14e 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index e587f76f5d3..0a32246ce06 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 1d007233120..e1c33c4b13b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/pom.xml b/pom.xml index 85ddb331552..b9b34479851 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.0 + 0.3.1-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 215d3bf0cbe..19553828205 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 049318700ef..09c44aea4b0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.0 + 0.3.1-SNAPSHOT From d1626576c069cc9ee84c113b0bba04ed25c91207 Mon Sep 17 00:00:00 2001 From: James Estes Date: Sat, 9 Feb 2013 22:33:00 -0700 Subject: [PATCH 28/42] Working toward making it easier to add new SegmentPullers. 1) Move the local cacheFile logic out of the S3 pullers into the SingleSegmentLoader 2) Make the S3SegmentPuller just pull down the file 3) Make the Loader do the unzip, ungzip, or rename 4) 2 and 3 make S3ZippedSegmentPuller not necessary (still there, just deprecated and empty) 4) Tweak the TaskToolbox so that the Pullers returned by getSegmentGetters behave the same as they did before --- .../druid/merger/common/TaskToolbox.java | 39 +++-- .../druid/initialization/ServerInit.java | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 144 +++++----------- .../druid/loading/S3ZippedSegmentPuller.java | 163 +----------------- .../metamx/druid/loading/SegmentPuller.java | 2 +- .../druid/loading/SingleSegmentLoader.java | 141 ++++++++++++++- 6 files changed, 219 insertions(+), 282 deletions(-) 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 0cebe1fc91c..d775921d560 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 @@ -21,13 +21,15 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.S3SegmentGetterConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentPuller; +import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.SingleSegmentLoader; +import com.metamx.druid.loading.StorageAdapterLoadingException; 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.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -88,19 +90,28 @@ public class TaskToolbox public Map getSegmentGetters(final Task task) { - final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig() - { - @Override - public File getCacheDirectory() - { - return new File(config.getTaskDir(task), "fetched_segments"); - } - }; + LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); return ImmutableMap.builder() - .put("s3", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_union", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig)) + .put("s3", puller) + .put("s3_union", puller) + .put("s3_zip", puller) .build(); } + + class LoaderPullerAdapter implements SegmentPuller{ + private SingleSegmentLoader loader; + public LoaderPullerAdapter(File cacheDir){ + loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); + } + @Override + public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { + return loader.getSegmentFiles(loadSpec); + } + + @Override + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + return -1; + } + } } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 1d727f9abe3..3a510e8b23c 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -35,7 +35,6 @@ import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -69,8 +68,8 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client, config); - final S3ZippedSegmentPuller zippedGetter = new S3ZippedSegmentPuller(s3Client, config); + final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { factory = new MMappedQueryableIndexFactory(); @@ -78,11 +77,12 @@ public class ServerInit throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); } + SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", new SingleSegmentLoader(segmentGetter, factory)) - .put("s3_zip", new SingleSegmentLoader(zippedGetter, factory)) - .build() + .put("s3", segmentLoader) + .put("s3_zip", segmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java index 380489548d5..f85a489b1fe 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java @@ -25,17 +25,15 @@ import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import org.joda.time.DateTime; import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; import java.util.Map; -import java.util.zip.GZIPInputStream; /** */ @@ -48,133 +46,85 @@ public class S3SegmentPuller implements SegmentPuller private static final String KEY = "key"; private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; @Inject public S3SegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config + RestS3Service s3Client ) { this.s3Client = s3Client; - this.config = config; } @Override public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); + S3Coords s3Coords = new S3Coords(segment); - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); + log.info("Loading index at path[%s]", s3Coords); - S3Object s3Obj = null; + if(!isObjectInBucket(s3Coords)){ + throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + long currTime = System.currentTimeMillis(); File tmpFile = null; + S3Object s3Obj = null; + try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); + log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile.getParentFile(); - } - FileUtils.deleteDirectory(cacheFile.getParentFile()); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); + log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); - } - cacheFile.delete(); - - if (s3Path.endsWith("gz")) { - log.info("Decompressing file[%s] to [%s]", tmpFile, cacheFile); - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(tmpFile)), - cacheFile - ); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } else { - log.info("Rename tmpFile[%s] to cacheFile[%s]", tmpFile, cacheFile); - if (!tmpFile.renameTo(cacheFile)) { - log.warn("Error renaming tmpFile[%s] to cacheFile[%s]. Copying instead.", tmpFile, cacheFile); - - StreamUtils.copyToFileAndClose(new FileInputStream(tmpFile), cacheFile); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - return cacheFile.getParentFile(); + return tmpFile; } catch (Exception e) { + if(tmpFile!=null && tmpFile.exists()){ + tmpFile.delete(); + } throw new StorageAdapterLoadingException(e, e.getMessage()); } finally { S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } } } - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return String.format( - "%s/%s", s3Bucket, s3Path.endsWith("gz") ? s3Path.substring(0, s3Path.length() - ".gz".length()) : s3Path - ); + private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } catch (ServiceException e) { + throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); + } } @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath(MapUtils.getString(loadSpec, BUCKET), MapUtils.getString(loadSpec, KEY)) - ); - + public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { + S3Coords coords = new S3Coords(segment); try { - final File parentFile = cacheFile.getParentFile(); - log.info("Recursively deleting file[%s]", parentFile); - FileUtils.deleteDirectory(parentFile); - } - catch (IOException e) { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } catch (S3ServiceException e) { throw new StorageAdapterLoadingException(e, e.getMessage()); } + } - return true; + private class S3Coords { + String bucket; + String path; + + public S3Coords(DataSegment segment) { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if(path.startsWith("/")){ + path = path.substring(1); + } + } + public String toString(){ + return String.format("s3://%s/%s", bucket, path); + } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java index 8fd8ebd4542..a3a7c724687 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java @@ -19,169 +19,14 @@ package com.metamx.druid.loading; -import com.google.common.io.Closeables; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** + * @deprecated */ -public class S3ZippedSegmentPuller implements SegmentPuller +public class S3ZippedSegmentPuller extends S3SegmentPuller { - private static final Logger log = new Logger(S3ZippedSegmentPuller.class); - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; - - public S3ZippedSegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config - ) - { - this.s3Client = s3Client; - this.config = config; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - - if (s3Path.startsWith("/")) { - s3Path = s3Path.substring(1); - } - - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); - - S3Object s3Obj = null; - File tmpFile = null; - try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } - - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile; - } - FileUtils.deleteDirectory(cacheFile); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); - - if (cacheFile.exists()) { - FileUtils.deleteDirectory(cacheFile); - } - cacheFile.mkdirs(); - - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(tmpFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - log.info("Deleting tmpFile[%s]", tmpFile); - tmpFile.delete(); - - return cacheFile; - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } - } - } - - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return new File(String.format("%s/%s", s3Bucket, s3Path)).getParent(); - } - - @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath( - MapUtils.getString(loadSpec, BUCKET), - MapUtils.getString(loadSpec, KEY) - ) - ); - - try { - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); - } - catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - - return true; + public S3ZippedSegmentPuller(RestS3Service s3Client) { + super(s3Client); } } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java index 9cba65f425c..3e5f1b1a161 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java @@ -29,5 +29,5 @@ import java.util.Map; public interface SegmentPuller { public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - public boolean cleanSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; + long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 19c3981e988..ae62cfda1e9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -19,40 +19,171 @@ package com.metamx.druid.loading; +import com.google.common.base.Joiner; +import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; + +import java.io.*; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; /** */ public class SingleSegmentLoader implements SegmentLoader { + private static final Logger log = new Logger(SingleSegmentLoader.class); + private final SegmentPuller segmentPuller; private final QueryableIndexFactory factory; + private File cacheDirectory; + private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory - ) + SegmentPuller segmentPuller, + QueryableIndexFactory factory, + File cacheDirectory) { this.segmentPuller = segmentPuller; this.factory = factory; + this.cacheDirectory = cacheDirectory; } @Override public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException { - final QueryableIndex index = factory.factorize(segmentPuller.getSegmentFiles(segment)); + File segmentFiles = getSegmentFiles(segment); + final QueryableIndex index = factory.factorize(segmentFiles); return new QueryableIndexSegment(segment.getIdentifier(), index); } + public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + File cacheFile = getCacheFile(segment); + if (cacheFile.exists()) { + long localLastModified = cacheFile.lastModified(); + long remoteLastModified = segmentPuller.getLastModified(segment); + if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + log.info( + "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", + cacheFile, + localLastModified, + remoteLastModified + ); + return cacheFile.getParentFile(); + } + } + + File pulledFile = segmentPuller.getSegmentFiles(segment); + + if(!cacheFile.getParentFile().mkdirs()){ + log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + } + if (cacheFile.exists()) { + cacheFile.delete(); + } + + if(pulledFile.getName().endsWith(".zip")){ + unzip(pulledFile, cacheFile.getParentFile()); + } else if(pulledFile.getName().endsWith(".gz")){ + gunzip(pulledFile, cacheFile); + } else { + moveToCache(pulledFile, cacheFile); + } + + return cacheFile.getParentFile(); + } + + private File getCacheFile(DataSegment segment) { + String outputKey = JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + + return new File(cacheDirectory, outputKey); + } + + private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); + if(!pulledFile.renameTo(cacheFile)){ + log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); + + try { + StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete pulledFile[%s].", pulledFile); + } + } + } + + private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); + ZipInputStream zipIn = null; + OutputStream out = null; + ZipEntry entry = null; + try { + zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); + while ((entry = zipIn.getNextEntry()) != null) { + out = new FileOutputStream(new File(cacheFile, entry.getName())); + IOUtils.copy(zipIn, out); + zipIn.closeEntry(); + Closeables.closeQuietly(out); + out = null; + } + } catch(IOException e) { + throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); + } + finally { + Closeables.closeQuietly(out); + Closeables.closeQuietly(zipIn); + } + } + + private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); + try { + StreamUtils.copyToFileAndClose( + new GZIPInputStream(new FileInputStream(pulledFile)), + cacheFile + ); + } catch (IOException e) { + throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); + } + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + @Override public void cleanup(DataSegment segment) throws StorageAdapterLoadingException { - segmentPuller.cleanSegmentFiles(segment); + File cacheFile = getCacheFile(segment).getParentFile(); + + try { + log.info("Deleting directory[%s]", cacheFile); + FileUtils.deleteDirectory(cacheFile); + } + catch (IOException e) { + throw new StorageAdapterLoadingException(e, e.getMessage()); + } } + } From dc3459d3f931e3d230a97f23da38152e947798b0 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Wed, 13 Feb 2013 19:15:49 -0600 Subject: [PATCH 29/42] 1) Initial commit of refactorings on top of housejester's to simplify the zipping and simplify the creation of other methods of loading data --- .../metamx/druid/utils/CompressionUtils.java | 90 ++++++++++ .../examples/RealtimeStandaloneMain.java | 6 +- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/merger/common/TaskToolbox.java | 48 ++--- .../common/index/YeOldePlumberSchool.java | 10 +- .../common/task/IndexGeneratorTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 10 +- .../http/IndexerCoordinatorNode.java | 8 +- .../druid/merger/worker/http/WorkerNode.java | 8 +- .../metamx/druid/realtime/RealtimeNode.java | 22 +-- .../druid/realtime/RealtimePlumberSchool.java | 12 +- .../druid/realtime/S3SegmentPusher.java | 6 +- .../druid/coordination/ServerManager.java | 14 +- .../druid/coordination/ZkCoordinator.java | 6 +- .../druid/initialization/ServerInit.java | 4 +- ...mentPuller.java => DataSegmentPuller.java} | 22 ++- ...mentPusher.java => DataSegmentPusher.java} | 2 +- .../loading/DelegatingSegmentLoader.java | 8 +- .../loading/MMappedQueryableIndexFactory.java | 8 +- .../druid/loading/QueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPuller.java | 170 ++++++++++++++++++ ...ntPusher.java => S3DataSegmentPusher.java} | 12 +- .../metamx/druid/loading/S3SegmentPuller.java | 130 -------------- .../druid/loading/S3ZippedSegmentPuller.java | 32 ---- .../metamx/druid/loading/SegmentLoader.java | 4 +- ...tion.java => SegmentLoadingException.java} | 6 +- .../druid/loading/SingleSegmentLoader.java | 102 ++++------- .../druid/coordination/ServerManagerTest.java | 15 +- .../druid/loading/NoopSegmentLoader.java | 4 +- 29 files changed, 410 insertions(+), 363 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/utils/CompressionUtils.java rename server/src/main/java/com/metamx/druid/loading/{SegmentPuller.java => DataSegmentPuller.java} (55%) rename server/src/main/java/com/metamx/druid/loading/{SegmentPusher.java => DataSegmentPusher.java} (96%) create mode 100644 server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusher.java => S3DataSegmentPusher.java} (96%) delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java rename server/src/main/java/com/metamx/druid/loading/{StorageAdapterLoadingException.java => SegmentLoadingException.java} (88%) diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java new file mode 100644 index 00000000000..2b87d0a866d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -0,0 +1,90 @@ +/* + * 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.utils; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.metamx.common.ISE; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; +import sun.misc.IOUtils; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +/** + */ +public class CompressionUtils +{ + private static final Logger log = new Logger(CompressionUtils.class); + + public static void unzip(File pulledFile, File outDir) throws IOException + { + if (!(outDir.exists() && outDir.isDirectory())) { + throw new ISE("outDir[%s] must exist and be a directory"); + } + + log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); + InputStream in = null; + try { + in = new BufferedInputStream(new FileInputStream(pulledFile)); + unzip(in, outDir); + } + finally { + Closeables.closeQuietly(in); + } + } + + public static void unzip(InputStream in, File outDir) throws IOException + { + ZipInputStream zipIn = new ZipInputStream(in); + + ZipEntry entry; + while ((entry = zipIn.getNextEntry()) != null) { + OutputStream out = null; + try { + out = new FileOutputStream(new File(outDir, entry.getName())); + ByteStreams.copy(zipIn, out); + zipIn.closeEntry(); + } + finally { + Closeables.closeQuietly(out); + } + } + } + + public static void gunzip(File pulledFile, File outDir) throws IOException + { + log.info("Gunzipping file[%s] to [%s]", pulledFile, outDir); + StreamUtils.copyToFileAndClose(new GZIPInputStream(new FileInputStream(pulledFile)), outDir); + if (!pulledFile.delete()) { + log.error("Could not delete tmpFile[%s].", pulledFile); + } + } + +} diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index ecdf2606a3b..92eb86cc801 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; @@ -72,8 +72,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index e936d481489..ca5b9f64fd4 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.realtime.MetadataUpdater; import com.metamx.druid.realtime.MetadataUpdaterConfig; import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.phonebook.PhoneBook; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; @@ -74,8 +74,8 @@ public class RealtimeStandaloneMain // dummyMetadataUpdater will not send updates to db because standalone demo has no db rn.setMetadataUpdater(dummyMetadataUpdater); - rn.setSegmentPusher( - new SegmentPusher() + rn.setDataSegmentPusher( + new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException 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 d775921d560..f77b08ce713 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 @@ -20,14 +20,13 @@ package com.metamx.druid.merger.common; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; -import com.metamx.druid.loading.SegmentPuller; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.emitter.service.ServiceEmitter; @@ -35,6 +34,7 @@ import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; +import java.util.List; import java.util.Map; /** @@ -45,14 +45,14 @@ public class TaskToolbox private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final RestS3Service s3Client; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher segmentPusher; private final ObjectMapper objectMapper; public TaskToolbox( IndexerCoordinatorConfig config, ServiceEmitter emitter, RestS3Service s3Client, - SegmentPusher segmentPusher, + DataSegmentPusher segmentPusher, ObjectMapper objectMapper ) { @@ -78,7 +78,7 @@ public class TaskToolbox return s3Client; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getSegmentPusher() { return segmentPusher; } @@ -88,30 +88,20 @@ public class TaskToolbox return objectMapper; } - public Map getSegmentGetters(final Task task) + public Map getSegments(final Task task, List segments) + throws SegmentLoadingException { - LoaderPullerAdapter puller = new LoaderPullerAdapter(new File(config.getTaskDir(task), "fetched_segments")); + final SingleSegmentLoader loader = new SingleSegmentLoader( + new S3DataSegmentPuller(s3Client), + new MMappedQueryableIndexFactory(), + new File(config.getTaskDir(task), "fetched_segments") + ); - return ImmutableMap.builder() - .put("s3", puller) - .put("s3_union", puller) - .put("s3_zip", puller) - .build(); - } - - class LoaderPullerAdapter implements SegmentPuller{ - private SingleSegmentLoader loader; - public LoaderPullerAdapter(File cacheDir){ - loader = new SingleSegmentLoader(new S3SegmentPuller(s3Client), new MMappedQueryableIndexFactory(), cacheDir); - } - @Override - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException { - return loader.getSegmentFiles(loadSpec); + Map retVal = Maps.newLinkedHashMap(); + for (DataSegment segment : segments) { + retVal.put(segment, loader.getSegmentFiles(segment)); } - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - return -1; - } + return retVal; } } 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 7d456d29e01..703dbe898f7 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 @@ -35,7 +35,7 @@ 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 com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.FireHydrant; @@ -61,7 +61,7 @@ public class YeOldePlumberSchool implements PlumberSchool { private final Interval interval; private final String version; - private final SegmentPusher segmentPusher; + private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -70,13 +70,13 @@ public class YeOldePlumberSchool implements PlumberSchool public YeOldePlumberSchool( @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, - @JacksonInject("segmentPusher") SegmentPusher segmentPusher, + @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, @JacksonInject("tmpSegmentDir") File tmpSegmentDir ) { this.interval = interval; this.version = version; - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; } @@ -149,7 +149,7 @@ public class YeOldePlumberSchool implements PlumberSchool .withVersion(version) .withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload)); - segmentPusher.push(fileToUpload, segmentToUpload); + dataSegmentPusher.push(fileToUpload, segmentToUpload); log.info( "Uploaded segment[%s]", 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 b3da8978bd3..e56b9c4967d 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 @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.index.YeOldePlumberSchool; @@ -36,7 +37,6 @@ import com.metamx.druid.realtime.Firehose; import com.metamx.druid.realtime.FirehoseFactory; import com.metamx.druid.realtime.Plumber; import com.metamx.druid.realtime.Schema; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.realtime.Sink; @@ -111,7 +111,7 @@ public class IndexGeneratorTask extends AbstractTask // We need to track published segments. final List pushedSegments = new CopyOnWriteArrayList(); - final SegmentPusher wrappedSegmentPusher = new SegmentPusher() + final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher() { @Override public DataSegment push(File file, DataSegment segment) throws IOException @@ -128,7 +128,7 @@ public class IndexGeneratorTask extends AbstractTask final Plumber plumber = new YeOldePlumberSchool( getInterval(), context.getVersion(), - wrappedSegmentPusher, + wrappedDataSegmentPusher, tmpDir ).findPlumber(schema, metrics); 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 21f6c1e6416..2cfec4e5d2a 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 @@ -31,13 +31,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; 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.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.loading.SegmentPuller; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.coordinator.TaskContext; @@ -147,13 +145,7 @@ public abstract class MergeTask extends AbstractTask // download segments to merge - final Map segmentGetters = toolbox.getSegmentGetters(this); - final Map gettedSegments = Maps.newHashMap(); - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type")); - gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment)); - } + final Map gettedSegments = toolbox.getSegments(this, segments); // merge files together final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); 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 7d075cbe9e8..ec5404086d9 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 @@ -49,9 +49,9 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -403,12 +403,12 @@ public class IndexerCoordinatorNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); } } 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 237ae09d854..a152f0f003f 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 @@ -36,6 +36,7 @@ import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -44,9 +45,8 @@ import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -290,12 +290,12 @@ public class WorkerNode extends RegisteringNode PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); - final SegmentPusher segmentPusher = new S3SegmentPusher( + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, configFactory.build(S3SegmentPusherConfig.class), jsonMapper ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, jsonMapper); + taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); } } 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 d8fc7ebeea7..536074d6247 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -46,9 +46,9 @@ import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.S3SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -86,7 +86,7 @@ public class RealtimeNode extends BaseServerNode private final Map injectablesMap = Maps.newLinkedHashMap(); private MetadataUpdater metadataUpdater = null; - private SegmentPusher segmentPusher = null; + private DataSegmentPusher dataSegmentPusher = null; private List fireDepartments = null; private ServerView view = null; @@ -117,10 +117,10 @@ public class RealtimeNode extends BaseServerNode return this; } - public RealtimeNode setSegmentPusher(SegmentPusher segmentPusher) + public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - Preconditions.checkState(this.segmentPusher == null, "Cannot set segmentPusher once it has already been set."); - this.segmentPusher = segmentPusher; + Preconditions.checkState(this.dataSegmentPusher == null, "Cannot set segmentPusher once it has already been set."); + this.dataSegmentPusher = dataSegmentPusher; return this; } @@ -144,10 +144,10 @@ public class RealtimeNode extends BaseServerNode return metadataUpdater; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getDataSegmentPusher() { initializeSegmentPusher(); - return segmentPusher; + return dataSegmentPusher; } public List getFireDepartments() @@ -220,7 +220,7 @@ public class RealtimeNode extends BaseServerNode } injectables.put("queryRunnerFactoryConglomerate", getConglomerate()); - injectables.put("segmentPusher", segmentPusher); + injectables.put("segmentPusher", dataSegmentPusher); injectables.put("metadataUpdater", metadataUpdater); injectables.put("serverView", view); injectables.put("serviceEmitter", getEmitter()); @@ -256,7 +256,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { - if (segmentPusher == null) { + if (dataSegmentPusher == null) { final Properties props = getProps(); final RestS3Service s3Client; try { @@ -271,7 +271,7 @@ public class RealtimeNode extends BaseServerNode throw Throwables.propagate(e); } - segmentPusher = new S3SegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); } } 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 47a4ef0f40f..70c12eaaa45 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -48,7 +48,7 @@ import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -95,7 +95,7 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile RejectionPolicyFactory rejectionPolicyFactory = null; private volatile QueryRunnerFactoryConglomerate conglomerate = null; - private volatile SegmentPusher segmentPusher = null; + private volatile DataSegmentPusher dataSegmentPusher = null; private volatile MetadataUpdater metadataUpdater = null; private volatile ServerView serverView = null; private ServiceEmitter emitter; @@ -130,9 +130,9 @@ public class RealtimePlumberSchool implements PlumberSchool } @JacksonInject("segmentPusher") - public void setSegmentPusher(SegmentPusher segmentPusher) + public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; } @JacksonInject("metadataUpdater") @@ -325,7 +325,7 @@ public class RealtimePlumberSchool implements PlumberSchool QueryableIndex index = IndexIO.loadIndex(mergedFile); - DataSegment segment = segmentPusher.push( + DataSegment segment = dataSegmentPusher.push( mergedFile, sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) ); @@ -512,7 +512,7 @@ public class RealtimePlumberSchool implements PlumberSchool private void verifyState() { Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(segmentPusher, "must specify a segmentPusher to do this action."); + Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action."); Preconditions.checkNotNull(metadataUpdater, "must specify a metadataUpdater to do this action."); Preconditions.checkNotNull(serverView, "must specify a serverView to do this action."); Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); 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 5a74b17e223..2e40c398bdc 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -20,16 +20,18 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible + * + * Exists in 0.2, can be removed from 0.3 on */ @Deprecated -public class S3SegmentPusher extends com.metamx.druid.loading.S3SegmentPusher implements SegmentPusher +public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPusher implements DataSegmentPusher { public S3SegmentPusher( RestS3Service s3Client, diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 3bde07908c7..38e7d1e4a39 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -30,7 +30,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.query.BySegmentQueryRunner; @@ -104,24 +104,24 @@ public class ServerManager implements QuerySegmentWalker } } - public void loadSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void loadSegment(final DataSegment segment) throws SegmentLoadingException { final Segment adapter; try { adapter = segmentLoader.getSegment(segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { try { segmentLoader.cleanup(segment); } - catch (StorageAdapterLoadingException e1) { + catch (SegmentLoadingException e1) { // ignore } throw e; } if (adapter == null) { - throw new StorageAdapterLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); } synchronized (lock) { @@ -139,7 +139,7 @@ public class ServerManager implements QuerySegmentWalker ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { log.info("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); - throw new StorageAdapterLoadingException("Segment already exists[%s]", segment.getIdentifier()); + throw new SegmentLoadingException("Segment already exists[%s]", segment.getIdentifier()); } loadedIntervals.add( @@ -154,7 +154,7 @@ public class ServerManager implements QuerySegmentWalker } } - public void dropSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void dropSegment(final DataSegment segment) throws SegmentLoadingException { String dataSource = segment.getDataSource(); synchronized (lock) { 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 57d64e0ba32..1951205975c 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; @@ -245,14 +245,14 @@ public class ZkCoordinator implements DataSegmentChangeHandler } catch (IOException e) { removeSegment(segment); - throw new StorageAdapterLoadingException( + throw new SegmentLoadingException( "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile ); } yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { log.error(e, "Failed to load segment[%s]", segment); emitter.emit( new AlertEvent.Builder().build( diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 3a510e8b23c..82a5f263608 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -28,7 +28,7 @@ import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; @@ -68,7 +68,7 @@ public class ServerInit { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client); + final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); final QueryableIndexFactory factory; if ("mmap".equals(config.getQueryableFactoryType())) { diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java similarity index 55% rename from server/src/main/java/com/metamx/druid/loading/SegmentPuller.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java index 3e5f1b1a161..b821c653a6e 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java @@ -22,12 +22,26 @@ package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; import java.io.File; -import java.util.Map; /** */ -public interface SegmentPuller +public interface DataSegmentPuller { - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - long getLastModified(DataSegment segment) throws StorageAdapterLoadingException; + /** + * Pull down segment files for the given DataSegment and put them in the given directory. + * + * @param segment The segment to pull down files for + * @param dir The directory to store the files in + * @throws SegmentLoadingException if there are any errors + */ + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException; + + /** + * Returns the last modified time of the given segment. + * + * @param segment The segment to check the last modified time for + * @return the last modified time in millis from the epoch + * @throws SegmentLoadingException if there are any errors + */ + public long getLastModified(DataSegment segment) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java index 3700215efc1..5369480d6b9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java @@ -24,7 +24,7 @@ import com.metamx.druid.client.DataSegment; import java.io.File; import java.io.IOException; -public interface SegmentPusher +public interface DataSegmentPusher { public DataSegment push(File file, DataSegment segment) throws IOException; } diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index d576e59ae82..0f8e1e7074f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -44,24 +44,24 @@ public class DelegatingSegmentLoader implements SegmentLoader } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { return getLoader(segment.getLoadSpec()).getSegment(segment); } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { getLoader(segment.getLoadSpec()).cleanup(segment); } - private SegmentLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException + private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException { String type = MapUtils.getString(loadSpec, "type"); SegmentLoader loader = loaderTypes.get(type); if (loader == null) { - throw new StorageAdapterLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); } return loader; } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 648813d62ac..9f8594a30d2 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -34,7 +34,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); @Override - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { if (! IndexIO.canBeMapped(parentDir)) { @@ -46,11 +46,11 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory IndexIO.storeLatest(IndexIO.readIndex(parentDir), canBeMappedDir); if (! IndexIO.canBeMapped(canBeMappedDir)) { - throw new StorageAdapterLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); + throw new SegmentLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); } for (File file : canBeMappedDir.listFiles()) { if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); + throw new SegmentLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); } } FileUtils.deleteDirectory(canBeMappedDir); @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index d7f60309aa6..276bbc2028a 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -27,5 +27,5 @@ import java.io.File; */ public interface QueryableIndexFactory { - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException; + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java new file mode 100644 index 00000000000..011e1633ca1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -0,0 +1,170 @@ +/* + * 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.loading; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.google.common.io.Files; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.common.s3.S3Utils; +import com.metamx.druid.utils.CompressionUtils; +import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; +import org.jets3t.service.model.S3Object; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + */ +public class S3DataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(S3DataSegmentPuller.class); + + private static final String BUCKET = "bucket"; + private static final String KEY = "key"; + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentPuller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + @Override + public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException + { + S3Coords s3Coords = new S3Coords(segment); + + log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); + + if (!isObjectInBucket(s3Coords)) { + throw new SegmentLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + + if (!outDir.isDirectory()) { + throw new ISE("outDir[%s] must be a directory.", outDir); + } + + long startTime = System.currentTimeMillis(); + S3Object s3Obj = null; + + try { + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + + InputStream in = null; + try { + in = s3Obj.getDataInputStream(); + final String key = s3Obj.getKey(); + if (key.endsWith(".zip")) { + CompressionUtils.unzip(in, outDir); + } else if (key.endsWith(".gz")) { + final File outFile = new File(outDir, toFilename(key, ".gz")); + ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); + } else { + ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); + } + log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + } + catch (IOException e) { + FileUtils.deleteDirectory(outDir); + throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + } + finally { + Closeables.closeQuietly(in); + } + } + catch (Exception e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + finally { + S3Utils.closeStreamsQuietly(s3Obj); + } + + } + + private String toFilename(String key, final String suffix) + { + String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/' + filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end + return filename; + } + + private boolean isObjectInBucket(S3Coords coords) throws SegmentLoadingException + { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + S3Coords coords = new S3Coords(segment); + try { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } + catch (S3ServiceException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + } + + private static class S3Coords + { + String bucket; + String path; + + public S3Coords(DataSegment segment) + { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if (path.startsWith("/")) { + path = path.substring(1); + } + } + + public String toString() + { + return String.format("s3://%s/%s", bucket, path); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 5af4b905719..89a15b056ec 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -41,19 +41,19 @@ import java.security.NoSuchAlgorithmException; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; -public class S3SegmentPusher implements SegmentPusher +public class S3DataSegmentPusher implements DataSegmentPusher { - private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); + private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; private final S3SegmentPusherConfig config; private final ObjectMapper jsonMapper; - public S3SegmentPusher( - RestS3Service s3Client, - S3SegmentPusherConfig config, - ObjectMapper jsonMapper + public S3DataSegmentPusher( + RestS3Service s3Client, + S3SegmentPusherConfig config, + ObjectMapper jsonMapper ) { this.s3Client = s3Client; diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java deleted file mode 100644 index f85a489b1fe..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * 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.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.File; -import java.util.Map; - -/** - */ -public class S3SegmentPuller implements SegmentPuller -{ - private static final Logger log = new Logger(S3SegmentPuller.class); - private static final long DEFAULT_TIMEOUT = 5 * 60 * 1000; - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - - @Inject - public S3SegmentPuller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - S3Coords s3Coords = new S3Coords(segment); - - log.info("Loading index at path[%s]", s3Coords); - - if(!isObjectInBucket(s3Coords)){ - throw new StorageAdapterLoadingException("IndexFile[%s] does not exist.", s3Coords); - } - - long currTime = System.currentTimeMillis(); - File tmpFile = null; - S3Object s3Obj = null; - - try { - s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); - tmpFile = File.createTempFile(s3Coords.bucket, new DateTime().toString() + s3Coords.path.replace('/', '_')); - log.info("Downloading file[%s] to local tmpFile[%s] for segment[%s]", s3Coords, tmpFile, segment); - - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", tmpFile, downloadEndTime - currTime); - - return tmpFile; - } - catch (Exception e) { - if(tmpFile!=null && tmpFile.exists()){ - tmpFile.delete(); - } - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - } - } - - private boolean isObjectInBucket(S3Coords coords) throws StorageAdapterLoadingException { - try { - return s3Client.isObjectInBucket(coords.bucket, coords.path); - } catch (ServiceException e) { - throw new StorageAdapterLoadingException(e, "Problem communicating with S3 checking bucket/path[%s]", coords); - } - } - - @Override - public long getLastModified(DataSegment segment) throws StorageAdapterLoadingException { - S3Coords coords = new S3Coords(segment); - try { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); - return objDetails.getLastModifiedDate().getTime(); - } catch (S3ServiceException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - } - - private class S3Coords { - String bucket; - String path; - - public S3Coords(DataSegment segment) { - Map loadSpec = segment.getLoadSpec(); - bucket = MapUtils.getString(loadSpec, BUCKET); - path = MapUtils.getString(loadSpec, KEY); - if(path.startsWith("/")){ - path = path.substring(1); - } - } - public String toString(){ - return String.format("s3://%s/%s", bucket, path); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java deleted file mode 100644 index a3a7c724687..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.loading; - -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -/** - * @deprecated - */ -public class S3ZippedSegmentPuller extends S3SegmentPuller -{ - public S3ZippedSegmentPuller(RestS3Service s3Client) { - super(s3Client); - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 1ca54b89106..20fa5592ac2 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -26,6 +26,6 @@ import com.metamx.druid.index.Segment; */ public interface SegmentLoader { - public Segment getSegment(DataSegment loadSpec) throws StorageAdapterLoadingException; - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException; + public Segment getSegment(DataSegment loadSpec) throws SegmentLoadingException; + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java similarity index 88% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java index d0f0ba4be93..d52fd6e3a82 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java @@ -21,9 +21,9 @@ package com.metamx.druid.loading; /** */ -public class StorageAdapterLoadingException extends Exception +public class SegmentLoadingException extends Exception { - public StorageAdapterLoadingException( + public SegmentLoadingException( String formatString, Object... objs ) @@ -31,7 +31,7 @@ public class StorageAdapterLoadingException extends Exception super(String.format(formatString, objs)); } - public StorageAdapterLoadingException( + public SegmentLoadingException( Throwable cause, String formatString, Object... objs diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index ae62cfda1e9..9aebfbd35f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,7 +20,6 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; -import com.google.common.io.Closeables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,12 +28,8 @@ import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; import java.io.*; -import java.util.zip.GZIPInputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; /** */ @@ -42,24 +37,25 @@ public class SingleSegmentLoader implements SegmentLoader { private static final Logger log = new Logger(SingleSegmentLoader.class); - private final SegmentPuller segmentPuller; + private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; private File cacheDirectory; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory, - File cacheDirectory) + DataSegmentPuller dataSegmentPuller, + QueryableIndexFactory factory, + File cacheDirectory + ) { - this.segmentPuller = segmentPuller; + this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; this.cacheDirectory = cacheDirectory; } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { File segmentFiles = getSegmentFiles(segment); final QueryableIndex index = factory.factorize(segmentFiles); @@ -67,43 +63,37 @@ public class SingleSegmentLoader implements SegmentLoader return new QueryableIndexSegment(segment.getIdentifier(), index); } - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException { + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { File cacheFile = getCacheFile(segment); if (cacheFile.exists()) { long localLastModified = cacheFile.lastModified(); - long remoteLastModified = segmentPuller.getLastModified(segment); - if(remoteLastModified > 0 && localLastModified >= remoteLastModified){ + long remoteLastModified = dataSegmentPuller.getLastModified(segment); + if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", cacheFile, localLastModified, remoteLastModified ); - return cacheFile.getParentFile(); + return cacheFile; } } - File pulledFile = segmentPuller.getSegmentFiles(segment); + dataSegmentPuller.getSegmentFiles(segment, cacheFile); - if(!cacheFile.getParentFile().mkdirs()){ + if (!cacheFile.getParentFile().mkdirs()) { log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); } if (cacheFile.exists()) { cacheFile.delete(); } - if(pulledFile.getName().endsWith(".zip")){ - unzip(pulledFile, cacheFile.getParentFile()); - } else if(pulledFile.getName().endsWith(".gz")){ - gunzip(pulledFile, cacheFile); - } else { - moveToCache(pulledFile, cacheFile); - } - - return cacheFile.getParentFile(); + return cacheFile; } - private File getCacheFile(DataSegment segment) { + private File getCacheFile(DataSegment segment) + { String outputKey = JOINER.join( segment.getDataSource(), String.format( @@ -118,15 +108,22 @@ public class SingleSegmentLoader implements SegmentLoader return new File(cacheDirectory, outputKey); } - private void moveToCache(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { + private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException + { log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); - if(!pulledFile.renameTo(cacheFile)){ + if (!pulledFile.renameTo(cacheFile)) { log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); try { StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem moving pulledFile[%s] to cache[%s]", pulledFile, cacheFile); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Problem moving pulledFile[%s] to cache[%s]", + pulledFile, + cacheFile + ); } if (!pulledFile.delete()) { log.error("Could not delete pulledFile[%s].", pulledFile); @@ -134,46 +131,8 @@ public class SingleSegmentLoader implements SegmentLoader } } - private void unzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Unzipping file[%s] to [%s]", pulledFile, cacheFile); - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(pulledFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } catch(IOException e) { - throw new StorageAdapterLoadingException(e,"Problem unzipping[%s]", pulledFile); - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - } - - private void gunzip(File pulledFile, File cacheFile) throws StorageAdapterLoadingException { - log.info("Gunzipping file[%s] to [%s]", pulledFile, cacheFile); - try { - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(pulledFile)), - cacheFile - ); - } catch (IOException e) { - throw new StorageAdapterLoadingException(e,"Problem gunzipping[%s]", pulledFile); - } - if (!pulledFile.delete()) { - log.error("Could not delete tmpFile[%s].", pulledFile); - } - } - @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { File cacheFile = getCacheFile(segment).getParentFile(); @@ -182,8 +141,7 @@ public class SingleSegmentLoader implements SegmentLoader FileUtils.deleteDirectory(cacheFile); } catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, e.getMessage()); } } - } 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 762662741a6..d10566c5da4 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -31,7 +31,6 @@ import com.metamx.common.Pair; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Capabilities; import com.metamx.druid.Druids; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; @@ -39,12 +38,9 @@ import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.ConcatQueryRunner; @@ -54,7 +50,6 @@ import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; @@ -62,7 +57,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -72,7 +66,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -101,7 +94,7 @@ public class ServerManagerTest } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { } @@ -245,7 +238,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } @@ -267,7 +260,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java index 29d784d3631..ca41c4dfec6 100644 --- a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -30,7 +30,7 @@ import org.joda.time.Interval; public class NoopSegmentLoader implements SegmentLoader { @Override - public Segment getSegment(final DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(final DataSegment segment) throws SegmentLoadingException { return new Segment() { @@ -61,7 +61,7 @@ public class NoopSegmentLoader implements SegmentLoader } @Override - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException { } } From f8c54a72c2b61319ee8cbdd7b15bceb132a82703 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Tue, 19 Feb 2013 19:22:59 -0600 Subject: [PATCH 30/42] 1) Changes to allow for local storage --- .../druid/jackson/DefaultObjectMapper.java | 2 +- .../metamx/druid/utils/CompressionUtils.java | 38 ++++++- .../examples/RealtimeStandaloneMain.java | 6 +- .../druid/indexer/DeterminePartitionsJob.java | 3 +- .../indexer/HadoopDruidIndexerConfig.java | 3 +- .../druid/indexer/IndexGeneratorJob.java | 3 +- .../druid/merger/common/TaskToolbox.java | 10 +- .../common/index/YeOldePlumberSchool.java | 21 +++- .../druid/merger/common/task/DeleteTask.java | 6 +- .../druid/merger/common/task/MergeTask.java | 5 +- .../http/IndexerCoordinatorNode.java | 4 +- .../druid/merger/worker/http/WorkerNode.java | 4 +- .../druid/realtime/FireDepartmentConfig.java | 2 + .../metamx/druid/realtime/RealtimeNode.java | 36 ++++-- .../druid/realtime/RealtimePlumberSchool.java | 15 ++- .../druid/realtime/S3SegmentPusher.java | 4 +- .../com/metamx/druid/http/ComputeNode.java | 4 +- .../druid/initialization/ServerInit.java | 22 ++-- .../druid/loading/DataSegmentPusherUtil.java | 44 ++++++++ .../druid/loading/LocalDataSegmentPuller.java | 105 ++++++++++++++++++ .../druid/loading/LocalDataSegmentPusher.java | 96 ++++++++++++++++ ...java => LocalDataSegmentPusherConfig.java} | 6 +- .../loading/MMappedQueryableIndexFactory.java | 2 +- .../druid/loading/S3DataSegmentPusher.java | 51 ++------- ...ig.java => S3DataSegmentPusherConfig.java} | 2 +- ...erConfig.java => SegmentLoaderConfig.java} | 13 ++- .../druid/loading/SingleSegmentLoader.java | 46 ++++---- 27 files changed, 433 insertions(+), 120 deletions(-) create mode 100644 server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java create mode 100644 server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java rename server/src/main/java/com/metamx/druid/loading/{S3SegmentGetterConfig.java => LocalDataSegmentPusherConfig.java} (86%) rename server/src/main/java/com/metamx/druid/loading/{S3SegmentPusherConfig.java => S3DataSegmentPusherConfig.java} (95%) rename server/src/main/java/com/metamx/druid/loading/{QueryableLoaderConfig.java => SegmentLoaderConfig.java} (76%) 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 84514bd6c4c..293f80d900d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -174,7 +174,7 @@ public class DefaultObjectMapper extends ObjectMapper configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); - configure(MapperFeature.AUTO_DETECT_CREATORS, false); +// configure(MapperFeature.AUTO_DETECT_CREATORS, false); https://github.com/FasterXML/jackson-databind/issues/170 configure(MapperFeature.AUTO_DETECT_FIELDS, false); configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); configure(MapperFeature.AUTO_DETECT_SETTERS, false); diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java index 2b87d0a866d..c34b8e7e960 100644 --- a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -21,6 +21,7 @@ package com.metamx.druid.utils; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Files; import com.metamx.common.ISE; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -29,6 +30,7 @@ import sun.misc.IOUtils; import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; @@ -36,6 +38,7 @@ import java.io.OutputStream; import java.util.zip.GZIPInputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; /** */ @@ -43,10 +46,43 @@ public class CompressionUtils { private static final Logger log = new Logger(CompressionUtils.class); + public static long zip(File directory, File outputZipFile) throws IOException + { + if (!directory.isDirectory()) { + throw new IOException(String.format("directory[%s] is not a directory", directory)); + } + + if (!outputZipFile.getName().endsWith(".zip")) { + log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory); + } + + long totalSize = 0; + ZipOutputStream zipOut = null; + try { + zipOut = new ZipOutputStream(new FileOutputStream(outputZipFile)); + File[] files = directory.listFiles(); + for (File file : files) { + log.info("Adding file[%s] with size[%,d]. Total size[%,d]", file, file.length(), totalSize); + if (file.length() >= Integer.MAX_VALUE) { + zipOut.close(); + outputZipFile.delete(); + throw new IOException(String.format("file[%s] too large [%,d]", file, file.length())); + } + zipOut.putNextEntry(new ZipEntry(file.getName())); + totalSize += ByteStreams.copy(Files.newInputStreamSupplier(file), zipOut); + } + } + finally { + Closeables.closeQuietly(zipOut); + } + + return totalSize; + } + public static void unzip(File pulledFile, File outDir) throws IOException { if (!(outDir.exists() && outDir.isDirectory())) { - throw new ISE("outDir[%s] must exist and be a directory"); + throw new ISE("outDir[%s] must exist and be a directory", outDir); } log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index ca5b9f64fd4..5f4d25cb95b 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -48,10 +48,12 @@ public class RealtimeStandaloneMain rn.setPhoneBook(dummyPhoneBook); MetadataUpdater dummyMetadataUpdater = - new MetadataUpdater(new DefaultObjectMapper(), + new MetadataUpdater( + new DefaultObjectMapper(), Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class), dummyPhoneBook, - null) { + null + ) { @Override public void publishSegment(DataSegment segment) throws IOException { 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 d4ee1941396..9a72d997987 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -266,8 +266,7 @@ public class DeterminePartitionsJob implements Jobby Context context ) throws IOException, InterruptedException { - // Create group key - // TODO -- There are more efficient ways to do this + // Create group key, there are probably more efficient ways of doing this final Map> dims = Maps.newTreeMap(); for(final String dim : inputRow.getDimensions()) { final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim)); 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 eed2339114b..979e2d989a4 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -662,8 +662,9 @@ public class HadoopDruidIndexerConfig return new Path( String.format( - "%s/%s_%s/%s/%s", + "%s/%s/%s_%s/%s/%s", getSegmentOutputDir(), + dataSource, bucketInterval.getStart().toString(), bucketInterval.getEnd().toString(), getVersion().toString(), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index d8eba264c11..0620ba2bc85 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -379,7 +379,8 @@ public class IndexGeneratorJob implements Jobby ); } else if (outputFS instanceof LocalFileSystem) { loadSpec = ImmutableMap.of( - "type", "test" + "type", "local", + "path", indexOutURI.getPath() ); } else { throw new ISE("Unknown file system[%s]", outputFS.getClass()); 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 f77b08ce713..0ab0cf49be6 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 @@ -25,6 +25,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.task.Task; @@ -94,7 +95,14 @@ public class TaskToolbox final SingleSegmentLoader loader = new SingleSegmentLoader( new S3DataSegmentPuller(s3Client), new MMappedQueryableIndexFactory(), - new File(config.getTaskDir(task), "fetched_segments") + new SegmentLoaderConfig() + { + @Override + public File getCacheDirectory() + { + return new File(config.getTaskDir(task), "fetched_segments"); + } + } ); Map retVal = Maps.newLinkedHashMap(); 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 703dbe898f7..c26888c4485 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 @@ -45,11 +45,11 @@ import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.Sink; - - +import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import java.io.File; +import java.io.IOException; import java.util.List; import java.util.Set; @@ -120,13 +120,13 @@ public class YeOldePlumberSchool implements PlumberSchool @Override public void finishJob() { + // The segment we will upload + File fileToUpload = null; + try { // User should have persisted everything by now. Preconditions.checkState(!theSink.swappable(), "All data must be persisted before fininshing the job!"); - // The segment we will upload - final File fileToUpload; - if(spilled.size() == 0) { throw new IllegalStateException("Nothing indexed?"); } else if(spilled.size() == 1) { @@ -160,6 +160,17 @@ public class YeOldePlumberSchool implements PlumberSchool log.warn(e, "Failed to merge and upload"); throw Throwables.propagate(e); } + finally { + try { + if (fileToUpload != null) { + log.info("Deleting Index File[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", fileToUpload); + } + } } private void spillIfSwappable() 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 5f37ad2853e..3e1bee62e5d 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 @@ -22,6 +22,7 @@ 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.google.common.io.Files; import com.metamx.common.logger.Logger; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; @@ -37,7 +38,7 @@ import com.metamx.druid.merger.coordinator.TaskContext; import com.metamx.druid.shard.NoneShardSpec; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -102,6 +103,9 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } } 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 2cfec4e5d2a..5d062b12892 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 @@ -46,7 +46,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.commons.codec.digest.DigestUtils; - +import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -168,6 +168,9 @@ public abstract class MergeTask extends AbstractTask emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + log.info("Deleting Uploaded Files[%s]", fileToUpload); + FileUtils.deleteDirectory(fileToUpload); + return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment)); } catch (Exception e) { 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 ec5404086d9..af29c7da2b2 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 @@ -51,7 +51,7 @@ import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -405,7 +405,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(config, emitter, s3Client, dataSegmentPusher, jsonMapper); 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 a152f0f003f..e71cf7f3e98 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 @@ -37,6 +37,7 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -45,7 +46,6 @@ import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusherConfig; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -292,7 +292,7 @@ public class WorkerNode extends RegisteringNode ); final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( s3Client, - configFactory.build(S3SegmentPusherConfig.class), + configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper ); taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, dataSegmentPusher, jsonMapper); 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 d98997b5051..efc0c7a598f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; @@ -31,6 +32,7 @@ public class FireDepartmentConfig private final int maxRowsInMemory; private final Period intermediatePersistPeriod; + @JsonCreator public FireDepartmentConfig( @JsonProperty("maxRowsInMemory") int maxRowsInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod 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 536074d6247..087e87d107f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -47,8 +47,10 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -258,20 +260,30 @@ public class RealtimeNode extends BaseServerNode { if (dataSegmentPusher == null) { final Properties props = getProps(); - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + dataSegmentPusher = new LocalDataSegmentPusher( + getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() ); } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } + else { - dataSegmentPusher = new S3DataSegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + dataSegmentPusher = new S3DataSegmentPusher( + s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() + ); + } } } 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 70c12eaaa45..775dc7d5305 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -307,7 +307,7 @@ public class RealtimePlumberSchool implements PlumberSchool } } - final File mergedFile; + File mergedFile = null; try { List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { @@ -337,6 +337,19 @@ public class RealtimePlumberSchool implements PlumberSchool .addData("interval", interval) .emit(); } + + + if (mergedFile != null) { + try { + if (mergedFile != null) { + log.info("Deleting Index File[%s]", mergedFile); + FileUtils.deleteDirectory(mergedFile); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", mergedFile); + } + } } } ); 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 2e40c398bdc..007ea188b82 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -21,7 +21,7 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -35,7 +35,7 @@ public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPushe { public S3SegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { 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 2230932d9a1..08b3eb93da3 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -39,7 +39,7 @@ import com.metamx.druid.coordination.ZkCoordinatorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.QueryableLoaderConfig; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; @@ -172,7 +172,7 @@ public class ComputeNode extends BaseServerNode ); setSegmentLoader( - ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(QueryableLoaderConfig.class)) + ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(SegmentLoaderConfig.class)) ); } catch (S3ServiceException e) { diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 82a5f263608..7cd6caf3c1b 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -26,15 +26,16 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; +import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.QueryableLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -63,26 +64,23 @@ public class ServerInit public static SegmentLoader makeDefaultQueryableLoader( RestS3Service s3Client, - QueryableLoaderConfig config + SegmentLoaderConfig config ) { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); + final QueryableIndexFactory factory = new MMappedQueryableIndexFactory(); - final QueryableIndexFactory factory; - if ("mmap".equals(config.getQueryableFactoryType())) { - factory = new MMappedQueryableIndexFactory(); - } else { - throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); - } + SingleSegmentLoader s3segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config); + SingleSegmentLoader localSegmentLoader = new SingleSegmentLoader(new LocalDataSegmentPuller(), factory, config); - SingleSegmentLoader segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config.getCacheDirectory()); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", segmentLoader) - .put("s3_zip", segmentLoader) - .build() + .put("s3", s3segmentLoader) + .put("s3_zip", s3segmentLoader) + .put("local", localSegmentLoader) + .build() ); return delegateLoader; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java new file mode 100644 index 00000000000..e72bd787bb3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -0,0 +1,44 @@ +/* + * 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.loading; + +import com.google.common.base.Joiner; +import com.metamx.druid.client.DataSegment; + +/** + */ +public class DataSegmentPusherUtil +{ + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + + public static String getStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java new file mode 100644 index 00000000000..8cdb8e0a7a2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java @@ -0,0 +1,105 @@ +/* + * 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.loading; + +import com.google.common.io.Files; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +/** + */ +public class LocalDataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(LocalDataSegmentPuller.class); + + @Override + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException + { + final File path = getFile(segment); + + if (path.isDirectory()) { + if (path.equals(dir)) { + log.info("Asked to load [%s] into itself, done!", dir); + return; + } + + log.info("Copying files from [%s] to [%s]", path, dir); + File file = null; + try { + final File[] files = path.listFiles(); + for (int i = 0; i < files.length; ++i) { + file = files[i]; + Files.copy(file, new File(dir, file.getName())); + } + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to copy file[%s].", file); + } + } else { + if (!path.getName().endsWith(".zip")) { + throw new SegmentLoadingException("File is not a zip file[%s]", path); + } + + log.info("Unzipping local file[%s] to [%s]", path, dir); + try { + CompressionUtils.unzip(path, dir); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to unzip file[%s]", path); + } + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + final File file = getFile(segment); + + long lastModified = Long.MAX_VALUE; + if (file.isDirectory()) { + for (File childFile : file.listFiles()) { + lastModified = Math.min(childFile.lastModified(), lastModified); + } + } + else { + lastModified = file.lastModified(); + } + + return lastModified; + } + + private File getFile(DataSegment segment) throws SegmentLoadingException + { + final Map loadSpec = segment.getLoadSpec(); + final File path = new File(MapUtils.getString(loadSpec, "path")); + + if (!path.exists()) { + throw new SegmentLoadingException("Asked to load path[%s], but it doesn't exist.", path); + } + + return path; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java new file mode 100644 index 00000000000..1493b162572 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -0,0 +1,96 @@ +/* + * 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.loading; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class LocalDataSegmentPusher implements DataSegmentPusher +{ + private static final Logger log = new Logger(LocalDataSegmentPusher.class); + + private final LocalDataSegmentPusherConfig config; + private final ObjectMapper jsonMapper; + + public LocalDataSegmentPusher( + LocalDataSegmentPusherConfig config, + ObjectMapper jsonMapper + ) + { + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException + { + File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + + if (dataSegmentFile.equals(outDir)) { + long size = 0; + for (File file : dataSegmentFile.listFiles()) { + size += file.length(); + } + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outDir)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + outDir.mkdirs(); + File outFile = new File(outDir, "index.zip"); + log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile); + long size = CompressionUtils.zip(dataSegmentFile, outFile); + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outFile)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException + { + File descriptorFile = new File(outDir, "descriptor.json"); + log.info("Creating descriptor file at[%s]", descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + return segment; + } + + private ImmutableMap makeLoadSpec(File outFile) + { + return ImmutableMap.of("type", "local", "path", outFile.toString()); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java similarity index 86% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java rename to server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java index c2a4c7f6308..d33a9a5130b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java @@ -25,8 +25,8 @@ import java.io.File; /** */ -public abstract class S3SegmentGetterConfig +public abstract class LocalDataSegmentPusherConfig { - @Config("druid.paths.indexCache") - public abstract File getCacheDirectory(); + @Config("druid.pusher.local.storageDirectory") + public abstract File getStorageDirectory(); } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 9f8594a30d2..9896c3f800b 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new SegmentLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, "%s", e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 89a15b056ec..273a07d36f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -22,24 +22,20 @@ 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; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; import com.metamx.emitter.EmittingLogger; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; - import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; -import java.io.*; +import java.io.File; +import java.io.IOException; import java.security.NoSuchAlgorithmException; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; public class S3DataSegmentPusher implements DataSegmentPusher { @@ -47,12 +43,12 @@ public class S3DataSegmentPusher implements DataSegmentPusher private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; - private final S3SegmentPusherConfig config; + private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; public S3DataSegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { @@ -67,35 +63,11 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Uploading [%s] to S3", indexFilesDir); String outputKey = JOINER.join( config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() + DataSegmentPusherUtil.getStorageDir(segment) ); - long indexSize = 0; final File zipOutFile = File.createTempFile("druid", "index.zip"); - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); - File[] indexFiles = indexFilesDir.listFiles(); - for (File indexFile : indexFiles) { - log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); - if (indexFile.length() >= Integer.MAX_VALUE) { - throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); - } - zipOut.putNextEntry(new ZipEntry(indexFile.getName())); - IOUtils.copy(new FileInputStream(indexFile), zipOut); - indexSize += indexFile.length(); - } - } - finally { - Closeables.closeQuietly(zipOut); - } + long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); try { S3Object toPush = new S3Object(zipOutFile); @@ -119,7 +91,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); - StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); @@ -128,9 +100,6 @@ public class S3DataSegmentPusher implements DataSegmentPusher log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); - log.info("Deleting Index File[%s]", indexFilesDir); - FileUtils.deleteDirectory(indexFilesDir); - log.info("Deleting zipped index File[%s]", zipOutFile); zipOutFile.delete(); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java similarity index 95% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index 0bd66a1a913..a2cada422fb 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class S3SegmentPusherConfig +public abstract class S3DataSegmentPusherConfig { @Config("druid.pusher.s3.bucket") public abstract String getBucket(); diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java similarity index 76% rename from server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index c6e2a72c931..294c91b9a38 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -21,13 +21,18 @@ package com.metamx.druid.loading; import org.skife.config.Config; +import java.io.File; + /** */ -public abstract class QueryableLoaderConfig extends S3SegmentGetterConfig +public abstract class SegmentLoaderConfig { - @Config("druid.queryable.factory") - public String getQueryableFactoryType() + @Config({"druid.paths.indexCache", "druid.segmentCache.path"}) + public abstract File getCacheDirectory(); + + @Config("druid.segmentCache.deleteOnRemove") + public boolean deleteOnRemove() { - return "mmap"; + return true; } } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 9aebfbd35f3..7e62f57fbf4 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -20,6 +20,7 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.inject.Inject; import com.metamx.common.StreamUtils; import com.metamx.common.logger.Logger; @@ -39,19 +40,19 @@ public class SingleSegmentLoader implements SegmentLoader private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; - private File cacheDirectory; + private final SegmentLoaderConfig config; private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( DataSegmentPuller dataSegmentPuller, QueryableIndexFactory factory, - File cacheDirectory + SegmentLoaderConfig config ) { this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; - this.cacheDirectory = cacheDirectory; + this.config = config; } @Override @@ -65,34 +66,37 @@ public class SingleSegmentLoader implements SegmentLoader public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment); - if (cacheFile.exists()) { - long localLastModified = cacheFile.lastModified(); + File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + long localLastModified = localStorageDir.lastModified(); long remoteLastModified = dataSegmentPuller.getLastModified(segment); if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { log.info( - "Found cacheFile[%s] with modified[%s], which is same or after remote[%s]. Using.", - cacheFile, - localLastModified, - remoteLastModified + "Found localStorageDir[%s] with modified[%s], which is same or after remote[%s]. Using.", + localStorageDir, localLastModified, remoteLastModified ); - return cacheFile; + return localStorageDir; } } - dataSegmentPuller.getSegmentFiles(segment, cacheFile); - - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); + if (localStorageDir.exists()) { + try { + FileUtils.deleteDirectory(localStorageDir); + } + catch (IOException e) { + log.warn(e, "Exception deleting previously existing local dir[%s]", localStorageDir); + } } - if (cacheFile.exists()) { - cacheFile.delete(); + if (!localStorageDir.mkdirs()) { + log.info("Unable to make parent file[%s]", localStorageDir); } - return cacheFile; + dataSegmentPuller.getSegmentFiles(segment, localStorageDir); + + return localStorageDir; } - private File getCacheFile(DataSegment segment) + private File getLocalStorageDir(DataSegment segment) { String outputKey = JOINER.join( segment.getDataSource(), @@ -105,7 +109,7 @@ public class SingleSegmentLoader implements SegmentLoader segment.getShardSpec().getPartitionNum() ); - return new File(cacheDirectory, outputKey); + return new File(config.getCacheDirectory(), outputKey); } private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException @@ -134,7 +138,7 @@ public class SingleSegmentLoader implements SegmentLoader @Override public void cleanup(DataSegment segment) throws SegmentLoadingException { - File cacheFile = getCacheFile(segment).getParentFile(); + File cacheFile = getLocalStorageDir(segment).getParentFile(); try { log.info("Deleting directory[%s]", cacheFile); From 606a2e4b01a60e2c0639d33a84565b839874edc8 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:49:55 -0600 Subject: [PATCH 31/42] [maven-release-plugin] prepare release druid-0.3.1 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 59cc716195c..22460ee95c5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/common/pom.xml b/common/pom.xml index 710c88d7274..2642875bc5f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 01078c607f4..1e5f30e4806 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1-SNAPSHOT + 0.3.1 com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/pom.xml b/examples/pom.xml index 4220ed243bc..75894e58a1a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ae5e1e767e9..b39ec8fdd15 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index c557c46800b..4d714682f71 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/index-common/pom.xml b/index-common/pom.xml index c67359cb14e..f9b00d18c1c 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0a32246ce06..deee2da0598 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/merger/pom.xml b/merger/pom.xml index e1c33c4b13b..cc73763841f 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/pom.xml b/pom.xml index b9b34479851..757862f4ba8 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1-SNAPSHOT + 0.3.1 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 19553828205..c65b9f7e4db 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 diff --git a/server/pom.xml b/server/pom.xml index 09c44aea4b0..c10ff22b4d2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1-SNAPSHOT + 0.3.1 From 699aadd2b19f028cc4280b6807e39a3d2c45a7b2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 15:50:01 -0600 Subject: [PATCH 32/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 22460ee95c5..453a44717a5 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 2642875bc5f..f813c06dca7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 1e5f30e4806..a2aa0136f94 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.1 + 0.3.2-SNAPSHOT com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 75894e58a1a..c2282b32a38 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index b39ec8fdd15..1b501a6239b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 4d714682f71..a79dccedb98 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index f9b00d18c1c..2140c83512f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index deee2da0598..0823e26e699 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index cc73763841f..e64b60808b1 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/pom.xml b/pom.xml index 757862f4ba8..ac1185a4a33 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.1 + 0.3.2-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c65b9f7e4db..f336b4edf12 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c10ff22b4d2..2bcf37d7054 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.1 + 0.3.2-SNAPSHOT From 5e123988a0c2b13ee1ae6dbde720c13c3e4b8398 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:37:51 -0600 Subject: [PATCH 33/42] 1) Specify old Jackson version to resolve version conflicts between AWS SDK and curator --- pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index ac1185a4a33..02001be5ce0 100644 --- a/pom.xml +++ b/pom.xml @@ -192,6 +192,16 @@ jackson-jaxrs-json-provider 2.1.3 + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + javax.inject javax.inject From 846bc0e4f0ee58997ef93f992b7197d20a1454a6 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:45 -0600 Subject: [PATCH 34/42] [maven-release-plugin] prepare release druid-0.3.2 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 453a44717a5..01fa2065885 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/common/pom.xml b/common/pom.xml index f813c06dca7..94c032684e4 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index a2aa0136f94..ac544f8a9eb 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2-SNAPSHOT + 0.3.2 com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/pom.xml b/examples/pom.xml index c2282b32a38..09bc6e84f53 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 1b501a6239b..ed856c5926b 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index a79dccedb98..b151a08c5ad 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/index-common/pom.xml b/index-common/pom.xml index 2140c83512f..1ff0704164f 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/indexer/pom.xml b/indexer/pom.xml index 0823e26e699..ffca56c7de8 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/merger/pom.xml b/merger/pom.xml index e64b60808b1..b4c22f1944a 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/pom.xml b/pom.xml index 02001be5ce0..19f95b109b3 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2-SNAPSHOT + 0.3.2 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f336b4edf12..da2ed45cbdb 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 diff --git a/server/pom.xml b/server/pom.xml index 2bcf37d7054..62600955d21 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2-SNAPSHOT + 0.3.2 From e0f6df1a5cc078d19d4ab3668bdfc0ab503c49cc Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Thu, 21 Feb 2013 20:39:50 -0600 Subject: [PATCH 35/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 01fa2065885..7109bc96949 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 94c032684e4..10dbd719368 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index ac544f8a9eb..4022fbca433 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.2 + 0.3.3-SNAPSHOT com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 09bc6e84f53..ca22ebde9b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index ed856c5926b..4cc3bc1514d 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index b151a08c5ad..15d7ec6b245 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index 1ff0704164f..55353f21508 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index ffca56c7de8..9652c64ed78 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index b4c22f1944a..08f98b6389b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/pom.xml b/pom.xml index 19f95b109b3..d24722c64cc 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.2 + 0.3.3-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index da2ed45cbdb..c9ed42132cf 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 62600955d21..5e73eab0396 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.2 + 0.3.3-SNAPSHOT From 8513a5ab2a59a925fca4763ccbc256bab7cfc0f2 Mon Sep 17 00:00:00 2001 From: Eric Tschetter Date: Sun, 24 Feb 2013 22:18:44 -0800 Subject: [PATCH 36/42] 1) Fix SimpleColumn to not produce NPEs when one of its parts is null. --- .../com/metamx/druid/index/column/SimpleColumn.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 93825a8e9f0..8179cd623e4 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -72,30 +72,30 @@ class SimpleColumn implements Column @Override public DictionaryEncodedColumn getDictionaryEncoding() { - return dictionaryEncodedColumn.get(); + return dictionaryEncodedColumn == null ? null : dictionaryEncodedColumn.get(); } @Override public RunLengthColumn getRunLengthColumn() { - return runLengthColumn.get(); + return runLengthColumn == null ? null : runLengthColumn.get(); } @Override public GenericColumn getGenericColumn() { - return genericColumn.get(); + return genericColumn == null ? null : genericColumn.get(); } @Override public ComplexColumn getComplexColumn() { - return complexColumn.get(); + return complexColumn == null ? null : complexColumn.get(); } @Override public BitmapIndex getBitmapIndex() { - return bitmapIndex.get(); + return bitmapIndex == null ? null : bitmapIndex.get(); } } From 7d7ce2b7feb98a8ec0f688e6312b0c88e6652192 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 10:57:26 -0800 Subject: [PATCH 37/42] Fix DeterminePartitionsJob ISE for dimensions not present in all rows --- .../druid/indexer/DeterminePartitionsJob.java | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 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 9a72d997987..f34ff2988f2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -393,6 +393,9 @@ public class DeterminePartitionsJob implements Jobby final Interval interval = maybeInterval.get(); final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8); + // Emit row-counter value. + write(context, groupKey, new DimValueCount("", "", 1)); + for(final Map.Entry> dimAndValues : dims.entrySet()) { final String dim = dimAndValues.getKey(); @@ -509,9 +512,23 @@ public class DeterminePartitionsJob implements Jobby Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); + final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); + final PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); - // "iterator" will take us over many candidate dimensions + log.info( + "Determining partitions for interval: %s", + config.getGranularitySpec().bucketInterval(bucket).orNull() + ); + + // First DVC should be the total row count indicator + final DimValueCount firstDvc = iterator.next(); + final int totalRows = firstDvc.numRows; + + if(!firstDvc.dim.equals("") || !firstDvc.value.equals("")) { + throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!"); + } + + // "iterator" will now take us over many candidate dimensions DimPartitions currentDimPartitions = null; DimPartition currentDimPartition = null; String currentDimPartitionStart = null; @@ -635,8 +652,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final int totalRows = dimPartitionss.values().iterator().next().getRows(); - int maxCardinality = Integer.MIN_VALUE; long minVariance = Long.MAX_VALUE; DimPartitions minVariancePartitions = null; @@ -644,12 +659,14 @@ public class DeterminePartitionsJob implements Jobby for(final DimPartitions dimPartitions : dimPartitionss.values()) { if(dimPartitions.getRows() != totalRows) { - throw new ISE( - "WTF?! Dimension[%s] row count %,d != expected row count %,d", + log.info( + "Dimension[%s] is not present in all rows (row count %,d != expected row count %,d)", dimPartitions.dim, dimPartitions.getRows(), totalRows ); + + continue; } // Make sure none of these shards are oversized @@ -683,7 +700,6 @@ public class DeterminePartitionsJob implements Jobby throw new ISE("No suitable partitioning dimension found!"); } - final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); final OutputStream out = Utils.makePathAndOutputStream( context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles() ); From 99c4f9446c399b9e64bbcbb51158dd3617623b2a Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:15 -0800 Subject: [PATCH 38/42] [maven-release-plugin] prepare release druid-0.3.3 --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 7109bc96949..0774bd0c214 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/common/pom.xml b/common/pom.xml index 10dbd719368..17b376a2ee5 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 4022fbca433..0902c3e8064 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3-SNAPSHOT + 0.3.3 com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/pom.xml b/examples/pom.xml index ca22ebde9b9..976379c6678 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 4cc3bc1514d..179061524db 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 15d7ec6b245..344f5718bc0 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/index-common/pom.xml b/index-common/pom.xml index 55353f21508..d191045860b 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/indexer/pom.xml b/indexer/pom.xml index 9652c64ed78..2fd9c17894b 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/merger/pom.xml b/merger/pom.xml index 08f98b6389b..55fb1c76d82 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/pom.xml b/pom.xml index d24722c64cc..a1f3ed99638 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3-SNAPSHOT + 0.3.3 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index c9ed42132cf..8ca33faab93 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 diff --git a/server/pom.xml b/server/pom.xml index 5e73eab0396..99f8ebcca4c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3-SNAPSHOT + 0.3.3 From 9964e7dfe446f6995a7fe8020084e5f4d07d277e Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 12:24:24 -0800 Subject: [PATCH 39/42] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- druid-services/pom.xml | 4 ++-- examples/pom.xml | 2 +- examples/rand/pom.xml | 2 +- examples/twitter/pom.xml | 2 +- index-common/pom.xml | 2 +- indexer/pom.xml | 2 +- merger/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 0774bd0c214..3af4d9b489a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 17b376a2ee5..ef9c33b67e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 0902c3e8064..b5b2a7e460c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.3.3 + 0.3.4-SNAPSHOT com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 976379c6678..ae597675a62 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 179061524db..b9ff5296fde 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 344f5718bc0..88f4beae7da 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/index-common/pom.xml b/index-common/pom.xml index d191045860b..34264feb745 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/indexer/pom.xml b/indexer/pom.xml index 2fd9c17894b..778f682a0f2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/merger/pom.xml b/merger/pom.xml index 55fb1c76d82..e39359574a5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/pom.xml b/pom.xml index a1f3ed99638..16d86ae8776 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.3.3 + 0.3.4-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 8ca33faab93..0e48bc168c0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 99f8ebcca4c..e373da72f3d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.3.3 + 0.3.4-SNAPSHOT From 14cf506c43d87e7c5ecb0d282a2478f7f049d5c4 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:01 -0800 Subject: [PATCH 40/42] DefaultObjectMapper: Add GuavaModule --- common/pom.xml | 8 ++++++-- .../com/metamx/druid/jackson/DefaultObjectMapper.java | 2 ++ pom.xml | 5 +++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/common/pom.xml b/common/pom.xml index ef9c33b67e1..03483c15caa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -80,8 +80,12 @@ jackson-databind - com.fasterxml.jackson.datatype - jackson-datatype-joda + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi 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 293f80d900d..12079a77959 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -36,6 +36,7 @@ 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.guava.GuavaModule; import com.fasterxml.jackson.datatype.joda.JodaModule; import com.google.common.base.Throwables; import com.metamx.common.Granularity; @@ -171,6 +172,7 @@ public class DefaultObjectMapper extends ObjectMapper } ); registerModule(serializerModule); + registerModule(new GuavaModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/pom.xml b/pom.xml index 16d86ae8776..1f1afee50e3 100644 --- a/pom.xml +++ b/pom.xml @@ -177,6 +177,11 @@ jackson-databind 2.1.4-mmx-2 + + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.1.2 + com.fasterxml.jackson.datatype jackson-datatype-joda From 2427e81874c117624fe9fc91af88e6e816d9368d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 25 Feb 2013 17:05:30 -0800 Subject: [PATCH 41/42] Merger: Feedback from code review --- .../common/actions/SegmentInsertAction.java | 23 +-------- .../common/actions/SegmentNukeAction.java | 23 +-------- .../common/actions/TaskActionToolbox.java | 48 +++++++++++++++++++ .../merger/common/config/TaskConfig.java | 2 +- .../merger/common/task/AbstractTask.java | 9 +--- .../task/IndexDeterminePartitionsTask.java | 12 ++++- .../common/task/IndexGeneratorTask.java | 14 +++++- .../druid/merger/common/task/IndexTask.java | 32 +++++++++---- .../druid/merger/common/task/MergeTask.java | 9 +++- .../merger/common/task/MergeTaskTest.java | 8 ++-- .../merger/common/task/TaskSerdeTest.java | 7 ++- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../merger/coordinator/TaskLifecycleTest.java | 8 ++-- 13 files changed, 123 insertions(+), 74 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 436c16e26fa..75ad4a9161f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -51,27 +51,8 @@ public class SegmentInsertAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-insert falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().equals(segment.getVersion()) - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, false)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index ca0d9f3a9bd..f1b61c58d9f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -51,27 +51,8 @@ public class SegmentNukeAction implements TaskAction @Override public Void perform(TaskActionToolbox toolbox) { - // Verify that each of these segments-to-nuke falls under some lock - // TODO: Should this be done while holding the giant lock on TaskLockbox? (To prevent anyone from grabbing - // TODO: these locks out from under us while the operation is ongoing.) Probably not necessary. - final List taskLocks = toolbox.getTaskLockbox().findLocksForTask(task); - for(final DataSegment segment : segments) { - final boolean ok = Iterables.any( - taskLocks, new Predicate() - { - @Override - public boolean apply(TaskLock taskLock) - { - return taskLock.getVersion().compareTo(segment.getVersion()) >= 0 - && taskLock.getDataSource().equals(segment.getDataSource()) - && taskLock.getInterval().contains(segment.getInterval()); - } - } - ); - - if(!ok) { - throw new ISE("No currently-held lock covers segment: %s", segment); - } + if(!toolbox.taskLockCoversSegments(task, segments, true)) { + throw new ISE("Segments not covered by locks for task: %s", task.getId()); } try { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java index 30f87a1f6c3..af8f6dcd40c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java @@ -1,10 +1,19 @@ package com.metamx.druid.merger.common.actions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.metamx.common.ISE; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.TaskLockbox; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.emitter.service.ServiceEmitter; +import java.util.List; +import java.util.Set; + public class TaskActionToolbox { private final TaskQueue taskQueue; @@ -44,4 +53,43 @@ public class TaskActionToolbox { return emitter; } + + public boolean taskLockCoversSegments( + final Task task, + final Set segments, + final boolean allowOlderVersions + ) + { + // Verify that each of these segments falls under some lock + + // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check + // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we + // NOTE: insert some segments from the task but not others. + + final List taskLocks = getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @Override + public boolean apply(TaskLock taskLock) + { + final boolean versionOk = allowOlderVersions + ? taskLock.getVersion().compareTo(segment.getVersion()) >= 0 + : taskLock.getVersion().equals(segment.getVersion()); + + return versionOk + && taskLock.getDataSource().equals(segment.getDataSource()) + && taskLock.getInterval().contains(segment.getInterval()); + } + } + ); + + if (!ok) { + return false; + } + } + + return true; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index c66009cd8ac..5b7609bd042 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -13,7 +13,7 @@ public abstract class TaskConfig @Config("druid.merger.rowFlushBoundary") @Default("500000") - public abstract long getRowFlushBoundary(); + public abstract int getDefaultRowFlushBoundary(); public File getTaskDir(final Task task) { return new File(getBaseTaskDir(), task.getId()); 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 53653aa6595..bb65225a6a5 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 @@ -69,20 +69,13 @@ public abstract class AbstractTask implements Task return dataSource; } + @JsonProperty("interval") @Override public Optional getFixedInterval() { return interval; } - // Awesome hack to get around lack of serde for Optional - // TODO Look into jackson-datatype-guava - @JsonProperty("interval") - private Interval getNullableIntervalForJackson() - { - return interval.orNull(); - } - @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { 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 b94625e7ffa..569aa3e8a29 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 @@ -50,11 +50,16 @@ public class IndexDeterminePartitionsTask extends AbstractTask { @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty private final Schema schema; + @JsonProperty private final long targetPartitionSize; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -63,7 +68,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("schema") Schema schema, - @JsonProperty("targetPartitionSize") long targetPartitionSize + @JsonProperty("targetPartitionSize") long targetPartitionSize, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -81,6 +87,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; this.targetPartitionSize = targetPartitionSize; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -244,7 +251,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask schema.getAggregators(), schema.getIndexGranularity(), shardSpec - ) + ), + rowFlushBoundary ); } } 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 514f6cb7461..f4daec2d90a 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 @@ -58,6 +58,9 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty private final Schema schema; + @JsonProperty + private final int rowFlushBoundary; + private static final Logger log = new Logger(IndexTask.class); @JsonCreator @@ -65,7 +68,8 @@ public class IndexGeneratorTask extends AbstractTask @JsonProperty("groupId") String groupId, @JsonProperty("interval") Interval interval, @JsonProperty("firehose") FirehoseFactory firehoseFactory, - @JsonProperty("schema") Schema schema + @JsonProperty("schema") Schema schema, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -83,6 +87,7 @@ public class IndexGeneratorTask extends AbstractTask this.firehoseFactory = firehoseFactory; this.schema = schema; + this.rowFlushBoundary = rowFlushBoundary; } @Override @@ -139,6 +144,11 @@ public class IndexGeneratorTask extends AbstractTask tmpDir ).findPlumber(schema, metrics); + // rowFlushBoundary for this job + final int myRowFlushBoundary = this.rowFlushBoundary > 0 + ? rowFlushBoundary + : toolbox.getConfig().getDefaultRowFlushBoundary(); + try { while(firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); @@ -157,7 +167,7 @@ public class IndexGeneratorTask extends AbstractTask int numRows = sink.add(inputRow); metrics.incrementProcessed(); - if(numRows >= toolbox.getConfig().getRowFlushBoundary()) { + if(numRows >= myRowFlushBoundary) { plumber.persist(firehose.commit()); } } else { 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 8cfaa15d300..246389470ef 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 @@ -42,11 +42,23 @@ import java.util.List; public class IndexTask extends AbstractTask { - @JsonProperty private final GranularitySpec granularitySpec; - @JsonProperty private final AggregatorFactory[] aggregators; - @JsonProperty private final QueryGranularity indexGranularity; - @JsonProperty private final long targetPartitionSize; - @JsonProperty private final FirehoseFactory firehoseFactory; + @JsonProperty + private final GranularitySpec granularitySpec; + + @JsonProperty + private final AggregatorFactory[] aggregators; + + @JsonProperty + private final QueryGranularity indexGranularity; + + @JsonProperty + private final long targetPartitionSize; + + @JsonProperty + private final FirehoseFactory firehoseFactory; + + @JsonProperty + private final int rowFlushBoundary; private static final Logger log = new Logger(IndexTask.class); @@ -57,7 +69,8 @@ public class IndexTask extends AbstractTask @JsonProperty("aggregators") AggregatorFactory[] aggregators, @JsonProperty("indexGranularity") QueryGranularity indexGranularity, @JsonProperty("targetPartitionSize") long targetPartitionSize, - @JsonProperty("firehose") FirehoseFactory firehoseFactory + @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { super( @@ -75,6 +88,7 @@ public class IndexTask extends AbstractTask this.indexGranularity = indexGranularity; this.targetPartitionSize = targetPartitionSize; this.firehoseFactory = firehoseFactory; + this.rowFlushBoundary = rowFlushBoundary; } public List toSubtasks() @@ -95,7 +109,8 @@ public class IndexTask extends AbstractTask indexGranularity, new NoneShardSpec() ), - targetPartitionSize + targetPartitionSize, + rowFlushBoundary ) ); } else { @@ -110,7 +125,8 @@ public class IndexTask extends AbstractTask aggregators, indexGranularity, new NoneShardSpec() - ) + ), + rowFlushBoundary ) ); } 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 164c883d18b..f317efc7040 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 @@ -22,6 +22,7 @@ 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.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Objects; @@ -33,6 +34,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -46,7 +48,6 @@ import com.metamx.druid.shard.NoneShardSpec; 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.joda.time.DateTime; import org.joda.time.Interval; @@ -281,7 +282,11 @@ public abstract class MergeTask extends AbstractTask ) ); - return String.format("%s_%s", dataSource, DigestUtils.sha1Hex(segmentIDs).toLowerCase()); + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); } private static Interval computeMergedInterval(final List segments) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index 5267b8c7983..0f1a7a66964 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -19,9 +19,10 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; import com.metamx.druid.client.DataSegment; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -72,11 +73,12 @@ public class MergeTaskTest @Test public void testID() { - final String desiredPrefix = "merge_foo_" + DigestUtils.sha1Hex( + final String desiredPrefix = "merge_foo_" + Hashing.sha1().hashString( "2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0" + "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" + "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0" - ) + "_"; + , Charsets.UTF_8 + ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, testMergeTask.getId().substring(0, desiredPrefix.length()) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java index 88b43a71667..51310bb2ef0 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -26,7 +26,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - null + null, + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -52,7 +53,8 @@ public class TaskSerdeTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, new NoneShardSpec() - ) + ), + -1 ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -97,6 +99,7 @@ public class TaskSerdeTest final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); + System.out.println(json); final Task task2 = jsonMapper.readValue(json, Task.class); Assert.assertEquals(task.getId(), task2.getId()); 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 b37ef435a41..5981bd8ed22 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 @@ -291,7 +291,7 @@ public class RemoteTaskRunnerTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 0; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index 3f469da6bb3..69a364e1900 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -106,7 +106,7 @@ public class TaskLifecycleTest } @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 50000; } @@ -182,7 +182,8 @@ public class TaskLifecycleTest IR("2010-01-02T01", "a", "b", 2), IR("2010-01-02T01", "a", "c", 1) ) - ) + ), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); @@ -216,7 +217,8 @@ public class TaskLifecycleTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, QueryGranularity.NONE, 10000, - newMockExceptionalFirehoseFactory() + newMockExceptionalFirehoseFactory(), + -1 ); final TaskStatus mergedStatus = runTask(indexTask); From d8fbddb9d418f71c316c9006a4b068d78788caaf Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 26 Feb 2013 11:36:07 -0800 Subject: [PATCH 42/42] Merger: Service discovery for worker -> master communication --- .../actions/RemoteTaskActionClient.java | 40 ++++++++++--- .../http/IndexerCoordinatorResource.java | 6 +- .../merger/worker/config/WorkerConfig.java | 3 + .../druid/merger/worker/http/WorkerNode.java | 58 ++++++++++++++++--- 4 files changed, 90 insertions(+), 17 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 2dc86ab3115..26900e29942 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -1,25 +1,31 @@ package com.metamx.druid.merger.common.actions; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; +import com.netflix.curator.x.discovery.ServiceInstance; +import com.netflix.curator.x.discovery.ServiceProvider; import java.net.URI; import java.net.URISyntaxException; +import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { private final HttpClient httpClient; + private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ObjectMapper jsonMapper) + public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { this.httpClient = httpClient; + this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; } @@ -34,20 +40,36 @@ public class RemoteTaskActionClient implements TaskActionClient .go(new ToStringResponseHandler(Charsets.UTF_8)) .get(); - // TODO Figure out how to check HTTP status code - if(response.equals("")) { - return null; - } else { - return jsonMapper.readValue(response, taskAction.getReturnTypeReference()); - } + final Map responseDict = jsonMapper.readValue( + response, + new TypeReference>() {} + ); + + return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); } catch (Exception e) { throw Throwables.propagate(e); } } - public URI getServiceUri() throws URISyntaxException + private URI getServiceUri() throws Exception { - return new URI("http://localhost:8087/mmx/merger/v1/action"); + final ServiceInstance instance = serviceProvider.getInstance(); + final String scheme; + final String host; + final int port; + final String path = "/mmx/merger/v1/action"; + + host = instance.getAddress(); + + if (instance.getSslPort() != null && instance.getSslPort() > 0) { + scheme = "https"; + port = instance.getSslPort(); + } else { + scheme = "http"; + port = instance.getPort(); + } + + return new URI(scheme, null, host, port, path, null, null); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index b01e27a9316..fd3e02bcad3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator.http; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -176,6 +177,9 @@ public class IndexerCoordinatorResource public Response doAction(final TaskAction action) { final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); - return Response.ok().entity(ret).build(); + final Map retMap = Maps.newHashMap(); + retMap.put("result", ret); + + return Response.ok().entity(retMap).build(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..51a11546d24 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,6 +39,9 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); + @Config("druid.worker.masterService") + public abstract String getMasterService(); + public int getCapacity() { return Runtime.getRuntime().availableProcessors() - 1; 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 06ba0f73a32..d02ffa5d9e3 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 @@ -35,6 +35,7 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentPusher; @@ -64,6 +65,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 com.netflix.curator.x.discovery.ServiceDiscovery; +import com.netflix.curator.x.discovery.ServiceProvider; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; @@ -105,6 +108,8 @@ public class WorkerNode extends RegisteringNode private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; + private ServiceDiscovery serviceDiscovery = null; + private ServiceProvider coordinatorServiceProvider = null; private WorkerCuratorCoordinator workerCuratorCoordinator = null; private TaskMonitor taskMonitor = null; private Server server = null; @@ -156,6 +161,18 @@ public class WorkerNode extends RegisteringNode return this; } + public WorkerNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) + { + this.coordinatorServiceProvider = coordinatorServiceProvider; + return this; + } + + public WorkerNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) + { + this.serviceDiscovery = serviceDiscovery; + return this; + } + public WorkerNode setWorkerCuratorCoordinator(WorkerCuratorCoordinator workerCuratorCoordinator) { this.workerCuratorCoordinator = workerCuratorCoordinator; @@ -175,10 +192,12 @@ public class WorkerNode extends RegisteringNode initializeS3Service(); initializeMonitors(); initializeMergerConfig(); + initializeCuratorFramework(); + initializeServiceDiscovery(); + initializeCoordinatorServiceProvider(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); - initializeCuratorFramework(); initializeCuratorCoordinator(); initializeTaskMonitor(); initializeServer(); @@ -328,7 +347,7 @@ public class WorkerNode extends RegisteringNode ); taskToolbox = new TaskToolbox( taskConfig, - new RemoteTaskActionClient(httpClient, jsonMapper), + new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, dataSegmentPusher, @@ -340,11 +359,36 @@ public class WorkerNode extends RegisteringNode public void initializeCuratorFramework() throws IOException { - final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); - curatorFramework = Initialization.makeCuratorFrameworkClient( - curatorConfig, - lifecycle - ); + if (curatorFramework == null) { + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); + curatorFramework = Initialization.makeCuratorFrameworkClient( + curatorConfig, + lifecycle + ); + } + } + + public void initializeServiceDiscovery() throws Exception + { + if (serviceDiscovery == null) { + final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, + config, + lifecycle + ); + } + } + + public void initializeCoordinatorServiceProvider() + { + if (coordinatorServiceProvider == null) { + this.coordinatorServiceProvider = Initialization.makeServiceProvider( + workerConfig.getMasterService(), + serviceDiscovery, + lifecycle + ); + } } public void initializeCuratorCoordinator()