1) Didn't remove the io.druid files from client. Remove those and make sure things compile

2) Switch DefaultObjectMapper to CommonObjectMapper
3) Create new DefaultObjectMapper in client that has Query stuff registered on it by default
This commit is contained in:
cheddar 2013-08-29 15:25:36 -05:00
parent c5760f43e4
commit 2a46086e20
58 changed files with 272 additions and 1032 deletions

View File

@ -12,7 +12,7 @@ import com.metamx.druid.aggregation.post.ConstantPostAggregator;
import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DefaultDimensionSpec;
import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.spec.DimensionSpec;
import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.AndDimFilter;
import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.DimFilter;
import com.metamx.druid.query.filter.NotDimFilter; import com.metamx.druid.query.filter.NotDimFilter;

View File

@ -1,137 +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;
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.segment.QuerySegmentWalker;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Duration;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
/**
*/
public abstract class BaseQuery<T> implements Query<T>
{
private final String dataSource;
private final Map<String, String> context;
private final QuerySegmentSpec querySegmentSpec;
private volatile Duration duration;
public BaseQuery(
String dataSource,
QuerySegmentSpec querySegmentSpec,
Map<String, String> context
)
{
Preconditions.checkNotNull(dataSource, "dataSource can't be null");
Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null");
this.dataSource = dataSource.toLowerCase();
this.context = context;
this.querySegmentSpec = querySegmentSpec;
}
@JsonProperty
@Override
public String getDataSource()
{
return dataSource;
}
@JsonProperty("intervals")
public QuerySegmentSpec getQuerySegmentSpec()
{
return querySegmentSpec;
}
@Override
public Sequence<T> run(QuerySegmentWalker walker)
{
return run(querySegmentSpec.lookup(this, walker));
}
public Sequence<T> run(QueryRunner<T> runner)
{
return runner.run(this);
}
@Override
public List<Interval> getIntervals()
{
return querySegmentSpec.getIntervals();
}
@Override
public Duration getDuration()
{
if (duration == null) {
Duration totalDuration = new Duration(0);
for (Interval interval : querySegmentSpec.getIntervals()) {
if (interval != null) {
totalDuration = totalDuration.plus(interval.toDuration());
}
}
duration = totalDuration;
}
return duration;
}
@JsonProperty
public Map<String, String> getContext()
{
return context;
}
@Override
public String getContextValue(String key)
{
return context == null ? null : context.get(key);
}
@Override
public String getContextValue(String key, String defaultValue)
{
String retVal = getContextValue(key);
return retVal == null ? defaultValue : retVal;
}
protected Map<String, String> computeOverridenContext(Map<String, String> overrides)
{
Map<String, String> overridden = Maps.newTreeMap();
final Map<String, String> context = getContext();
if (context != null) {
overridden.putAll(context);
}
overridden.putAll(overrides);
return overridden;
}
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 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.guice; package com.metamx.druid.guice;
import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.Module;
@ -8,8 +27,8 @@ import com.metamx.druid.http.log.EmittingRequestLoggerProvider;
import com.metamx.druid.http.log.FileRequestLoggerProvider; import com.metamx.druid.http.log.FileRequestLoggerProvider;
import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.http.log.RequestLogger;
import com.metamx.druid.http.log.RequestLoggerProvider; import com.metamx.druid.http.log.RequestLoggerProvider;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import io.druid.query.QuerySegmentWalker;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -25,11 +25,11 @@ import com.metamx.druid.client.CachingClusteredClient;
import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.FinalizeResultsQueryRunner;
import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.MetricsEmittingQueryRunner;
import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.QueryToolChestWarehouse;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval; import org.joda.time.Interval;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,9 +22,9 @@ package com.metamx.druid.http;
import com.metamx.druid.client.DirectDruidClient; import com.metamx.druid.client.DirectDruidClient;
import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.FinalizeResultsQueryRunner;
import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.QueryToolChestWarehouse;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval; import org.joda.time.Interval;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -32,11 +32,11 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Json;
import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.guice.annotations.Smile;
import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.http.log.RequestLogger;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QuerySegmentWalker;
import org.eclipse.jetty.server.Request; import org.eclipse.jetty.server.Request;
import org.joda.time.DateTime; import org.joda.time.DateTime;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -17,13 +17,22 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/ */
package io.druid.query; package com.metamx.druid.jackson;
import com.metamx.common.guava.Sequence; import com.fasterxml.jackson.core.JsonFactory;
/** /**
*/ */
public interface QueryRunner<T> public class DefaultObjectMapper extends CommonObjectMapper
{ {
public Sequence<T> run(Query<T> query); public DefaultObjectMapper()
{
this(null);
}
public DefaultObjectMapper(JsonFactory factory)
{
super(factory);
registerModule(new QueryRegisteringModule());
}
} }

View File

@ -0,0 +1,85 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 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.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.metamx.druid.query.dimension.DefaultDimensionSpec;
import com.metamx.druid.query.dimension.ExtractionDimensionSpec;
import com.metamx.druid.query.dimension.LegacyDimensionSpec;
import com.metamx.druid.query.extraction.PartialDimExtractionFn;
import com.metamx.druid.query.extraction.RegexDimExtractionFn;
import com.metamx.druid.query.extraction.SearchQuerySpecDimExtractionFn;
import com.metamx.druid.query.extraction.TimeDimExtractionFn;
import com.metamx.druid.query.group.GroupByQuery;
import com.metamx.druid.query.metadata.SegmentMetadataQuery;
import com.metamx.druid.query.search.SearchQuery;
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
import com.metamx.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.Query;
import io.druid.query.spec.DimExtractionFn;
import io.druid.query.spec.DimensionSpec;
/**
*/
public class QueryRegisteringModule extends SimpleModule
{
public QueryRegisteringModule()
{
super("QueryRegistering");
setMixInAnnotation(Query.class, QueriesMixin.class);
setMixInAnnotation(DimensionSpec.class, DimensionSpecMixin.class);
setMixInAnnotation(DimExtractionFn.class, DimensionSpecMixin.class);
}
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "timeseries", value = TimeseriesQuery.class),
@JsonSubTypes.Type(name = "search", value = SearchQuery.class),
@JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryQuery.class),
@JsonSubTypes.Type(name = "groupBy", value = GroupByQuery.class),
@JsonSubTypes.Type(name = "segmentMetadata", value = SegmentMetadataQuery.class)
})
public static interface QueriesMixin
{
}
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class),
@JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class)
})
public static interface DimensionSpecMixin
{
}
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class),
@JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class),
@JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class),
@JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class)
})
public static interface DimExtractionFnMixin
{
}
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,7 +21,8 @@ package com.metamx.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.query.spec.DimExtractionFn;
import io.druid.query.spec.DimensionSpec;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;

View File

@ -1,39 +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.query.dimension;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.druid.query.extraction.DimExtractionFn;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class),
@JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class)
})
public interface DimensionSpec
{
public String getDimension();
public String getOutputName();
public DimExtractionFn getDimExtractionFn();
public byte[] getCacheKey();
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,7 +21,8 @@ package com.metamx.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.query.spec.DimExtractionFn;
import io.druid.query.spec.DimensionSpec;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;

View File

@ -1,38 +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.query.extraction;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class),
@JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class),
@JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class),
@JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class)
})
public interface DimExtractionFn
{
public byte[] getCacheKey();
public String apply(String dimValue);
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,6 +21,7 @@ package com.metamx.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.query.spec.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.regex.Matcher; import java.util.regex.Matcher;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,6 +21,7 @@ package com.metamx.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.query.spec.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.regex.Matcher; import java.util.regex.Matcher;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.query.search.SearchQuerySpec; import com.metamx.druid.query.search.SearchQuerySpec;
import io.druid.query.spec.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.ibm.icu.text.SimpleDateFormat; import com.ibm.icu.text.SimpleDateFormat;
import io.druid.query.spec.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.text.ParseException; import java.text.ParseException;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,7 +22,7 @@ package com.metamx.druid.query.filter;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.query.spec.DimExtractionFn;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -30,12 +30,10 @@ import com.google.common.collect.Lists;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import com.metamx.druid.BaseQuery;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.Queries; import com.metamx.druid.query.Queries;
import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DefaultDimensionSpec;
import com.metamx.druid.query.dimension.DimensionSpec;
import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.DimFilter;
import com.metamx.druid.query.group.having.HavingSpec; import com.metamx.druid.query.group.having.HavingSpec;
import com.metamx.druid.query.group.orderby.DefaultLimitSpec; import com.metamx.druid.query.group.orderby.DefaultLimitSpec;
@ -43,8 +41,9 @@ import com.metamx.druid.query.group.orderby.LimitSpec;
import com.metamx.druid.query.group.orderby.NoopLimitSpec; import com.metamx.druid.query.group.orderby.NoopLimitSpec;
import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import com.metamx.druid.query.group.orderby.OrderByColumnSpec;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.query.Query; import io.druid.query.BaseQuery;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.LegacySegmentSpec;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
@ -210,7 +209,7 @@ public class GroupByQuery extends BaseQuery<Row>
@Override @Override
public String getType() public String getType()
{ {
return Query.GROUP_BY; return "groupBy";
} }
public Sequence<Row> applyLimit(Sequence<Row> results) public Sequence<Row> applyLimit(Sequence<Row> results)

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -36,7 +36,6 @@ import com.metamx.druid.index.v1.IncrementalIndex;
import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.MapBasedRow;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.input.Rows; import com.metamx.druid.input.Rows;
import com.metamx.druid.query.dimension.DimensionSpec;
import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.query.Query; import io.druid.query.Query;
@ -44,6 +43,7 @@ import io.druid.query.QueryRunner;
import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChest;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.query.spec.DimensionSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Minutes; import org.joda.time.Minutes;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -33,8 +33,8 @@ import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Sequences;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.dimension.DimensionSpec;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -25,8 +25,8 @@ import com.google.common.base.Function;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.dimension.DimensionSpec;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import java.util.List; import java.util.List;

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 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.query.group.orderby; package com.metamx.druid.query.group.orderby;
import com.google.common.base.Function; import com.google.common.base.Function;
@ -5,8 +24,8 @@ import com.google.common.base.Functions;
import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequence;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.dimension.DimensionSpec;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import java.util.List; import java.util.List;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,7 +21,7 @@ package com.metamx.druid.query.metadata;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.BaseQuery; import io.druid.query.BaseQuery;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
@ -69,7 +69,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
@Override @Override
public String getType() public String getType()
{ {
return Query.SEGMENT_METADATA; return "segmentMetadata";
} }
@Override @Override

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -24,12 +24,11 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.druid.BaseQuery;
import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.DimFilter;
import com.metamx.druid.result.Result; import com.metamx.druid.result.Result;
import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.result.SearchResultValue;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.query.Query; import io.druid.query.BaseQuery;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -91,7 +90,7 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
@Override @Override
public String getType() public String getType()
{ {
return Query.SEARCH; return "search";
} }
@Override @Override

View File

@ -1,47 +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.query.segment;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval;
/**
*/
public interface QuerySegmentWalker
{
/**
* Gets the Queryable for a given interval, the Queryable returned can be any version(s) or partitionNumber(s)
* such that it represents the interval.
*
* @param intervals the intervals to find a Queryable for
* @return a Queryable object that represents the interval
*/
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals);
/**
* Gets the Queryable for a given list of SegmentSpecs.
* exist.
*
* @return the Queryable object with the given SegmentSpecs
*/
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs);
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -21,6 +21,7 @@ package com.metamx.druid.query.segment;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Interval; import org.joda.time.Interval;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -24,10 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.druid.BaseQuery;
import com.metamx.druid.result.Result; import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.druid.result.TimeBoundaryResultValue;
import io.druid.query.Query; import io.druid.query.BaseQuery;
import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
@ -74,7 +73,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
@Override @Override
public String getType() public String getType()
{ {
return Query.TIME_BOUNDARY; return "timeBoundary";
} }
@Override @Override

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -23,14 +23,13 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.metamx.druid.BaseQuery;
import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.query.Queries; import com.metamx.druid.query.Queries;
import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.DimFilter;
import com.metamx.druid.result.Result; import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.druid.result.TimeseriesResultValue;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
import io.druid.query.Query; import io.druid.query.BaseQuery;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;
@ -76,7 +75,7 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
@Override @Override
public String getType() public String getType()
{ {
return Query.TIMESERIES; return "timeseries";
} }
@JsonProperty("filter") @JsonProperty("filter")

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 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.sql; package com.metamx.druid.sql;
import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.core.type.TypeReference;
@ -11,7 +30,6 @@ import com.google.common.io.Closeables;
import com.metamx.druid.Druids; import com.metamx.druid.Druids;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.query.dimension.DimensionSpec;
import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQuery;
import com.metamx.druid.result.Result; import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.druid.result.TimeseriesResultValue;
@ -19,6 +37,7 @@ import com.metamx.druid.sql.antlr4.DruidSQLLexer;
import com.metamx.druid.sql.antlr4.DruidSQLParser; import com.metamx.druid.sql.antlr4.DruidSQLParser;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import org.antlr.v4.runtime.ANTLRInputStream; import org.antlr.v4.runtime.ANTLRInputStream;
import org.antlr.v4.runtime.CharStream; import org.antlr.v4.runtime.CharStream;
import org.antlr.v4.runtime.CommonTokenStream; import org.antlr.v4.runtime.CommonTokenStream;

View File

@ -1,39 +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 io.druid.query;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.metamx.common.guava.Sequence;
/**
*/
public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
{
public byte[] computeCacheKey(QueryType query);
public TypeReference<CacheType> getCacheObjectClazz();
public Function<T, CacheType> prepareForCache();
public Function<CacheType, T> pullFromCache();
public Sequence<T> mergeSequences(Sequence<Sequence<T>> seqOfSequences);
}

View File

@ -1,75 +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 io.druid.query;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.guava.Sequence;
import com.metamx.druid.query.group.GroupByQuery;
import com.metamx.druid.query.metadata.SegmentMetadataQuery;
import com.metamx.druid.query.search.SearchQuery;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
import com.metamx.druid.query.timeseries.TimeseriesQuery;
import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Duration;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
@JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class),
@JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class),
@JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class),
@JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class)
})
public interface Query<T>
{
public static final String TIMESERIES = "timeseries";
public static final String SEARCH = "search";
public static final String TIME_BOUNDARY = "timeBoundary";
public static final String GROUP_BY = "groupBy";
public static final String SEGMENT_METADATA = "segmentMetadata";
public String getDataSource();
public boolean hasFilters();
public String getType();
public Sequence<T> run(QuerySegmentWalker walker);
public Sequence<T> run(QueryRunner<T> runner);
public List<Interval> getIntervals();
public Duration getDuration();
public String getContextValue(String key);
public String getContextValue(String key, String defaultValue);
public Query<T> withOverriddenContext(Map<String, String> contextOverride);
public Query<T> withQuerySegmentSpec(QuerySegmentSpec spec);
}

View File

@ -1,66 +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 io.druid.query;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.metamx.common.guava.Sequence;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.aggregation.MetricManipulationFn;
import io.druid.segment.LogicalSegment;
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
* maintenance burden when upgrading versions.
*/
public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultType>>
{
public abstract QueryRunner<ResultType> mergeResults(QueryRunner<ResultType> runner);
/**
* This method doesn't belong here, but it's here for now just to make it work.
*
* @param seqOfSequences
* @return
*/
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
public abstract Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
public abstract TypeReference<ResultType> getResultTypeReference();
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query) {
return null;
}
public QueryRunner<ResultType> preMergeQueryDecoration(QueryRunner<ResultType> runner) {
return runner;
}
public QueryRunner<ResultType> postMergeQueryDecoration(QueryRunner<ResultType> runner) {
return runner;
}
public <T extends LogicalSegment> List<T> filterSegments(QueryType query, List<T> segments) {
return segments;
}
}

View File

@ -1,106 +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 io.druid.query;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
/**
*/
public class SegmentDescriptor
{
private final Interval interval;
private final String version;
private final int partitionNumber;
@JsonCreator
public SegmentDescriptor(
@JsonProperty("itvl") Interval interval,
@JsonProperty("ver") String version,
@JsonProperty("part") int partitionNumber)
{
this.interval = interval;
this.version = version;
this.partitionNumber = partitionNumber;
}
@JsonProperty("itvl")
public Interval getInterval()
{
return interval;
}
@JsonProperty("ver")
public String getVersion()
{
return version;
}
@JsonProperty("part")
public int getPartitionNumber()
{
return partitionNumber;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SegmentDescriptor that = (SegmentDescriptor) o;
if (partitionNumber != that.partitionNumber) {
return false;
}
if (interval != null ? !interval.equals(that.interval) : that.interval != null) {
return false;
}
if (version != null ? !version.equals(that.version) : that.version != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = interval != null ? interval.hashCode() : 0;
result = 31 * result + (version != null ? version.hashCode() : 0);
result = 31 * result + partitionNumber;
return result;
}
@Override
public String toString()
{
return "SegmentDescriptor{" +
"interval=" + interval +
", version='" + version + '\'' +
", partitionNumber=" + partitionNumber +
'}';
}
}

View File

@ -1,27 +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 io.druid.query.aggregation;
/**
*/
public interface MetricManipulationFn
{
public Object manipulate(AggregatorFactory factory, Object object);
}

View File

@ -1,69 +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 io.druid.query.spec;
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.joda.time.Interval;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
*/
public class LegacySegmentSpec extends MultipleIntervalSegmentSpec
{
private static List<Interval> convertValue(Object intervals)
{
final List<?> intervalStringList;
if (intervals instanceof String) {
intervalStringList = Arrays.asList((((String) intervals).split(",")));
} else if (intervals instanceof Map) {
intervalStringList = (List) ((Map) intervals).get("intervals");
} else if (intervals instanceof List) {
intervalStringList = (List) intervals;
} else {
throw new IAE("Unknown type[%s] for intervals[%s]", intervals.getClass(), intervals);
}
return Lists.transform(
intervalStringList,
new Function<Object, Interval>()
{
@Override
public Interval apply(Object input)
{
return new Interval(input);
}
}
);
}
@JsonCreator
public LegacySegmentSpec(
Object intervals
)
{
super(convertValue(intervals));
}
}

View File

@ -1,67 +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 io.druid.query.spec;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.utils.JodaUtils;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import org.joda.time.Interval;
import java.util.Collections;
import java.util.List;
/**
*/
public class MultipleIntervalSegmentSpec implements QuerySegmentSpec
{
private final List<Interval> intervals;
@JsonCreator
public MultipleIntervalSegmentSpec(
@JsonProperty("intervals") List<Interval> intervals
)
{
this.intervals = Collections.unmodifiableList(JodaUtils.condenseIntervals(intervals));
}
@Override
@JsonProperty("intervals")
public List<Interval> getIntervals()
{
return intervals;
}
@Override
public <T> QueryRunner<T> lookup(Query<T> query, QuerySegmentWalker walker)
{
return walker.getQueryRunnerForIntervals(query, intervals);
}
@Override
public String toString()
{
return getClass().getSimpleName() + "{" +
"intervals=" + intervals +
'}';
}
}

View File

@ -1,94 +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 io.druid.query.spec;
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.segment.QuerySegmentWalker;
import com.metamx.druid.utils.JodaUtils;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval;
import java.util.List;
/**
*/
public class MultipleSpecificSegmentSpec implements QuerySegmentSpec
{
private final List<SegmentDescriptor> descriptors;
private volatile List<Interval> intervals = null;
@JsonCreator
public MultipleSpecificSegmentSpec(
@JsonProperty("segments") List<SegmentDescriptor> descriptors
)
{
this.descriptors = descriptors;
}
@JsonProperty("segments")
public List<SegmentDescriptor> getDescriptors()
{
return descriptors;
}
@Override
public List<Interval> getIntervals()
{
if (intervals != null) {
return intervals;
}
intervals = JodaUtils.condenseIntervals(
Iterables.transform(
descriptors,
new Function<SegmentDescriptor, Interval>()
{
@Override
public Interval apply(SegmentDescriptor input)
{
return input.getInterval();
}
}
)
);
return intervals;
}
@Override
public <T> QueryRunner<T> lookup(Query<T> query, QuerySegmentWalker walker)
{
return walker.getQueryRunnerForSegments(query, descriptors);
}
@Override
public String toString()
{
return "MultipleSpecificSegmentSpec{" +
"descriptors=" + descriptors +
'}';
}
}

View File

@ -1,43 +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 io.druid.query.spec;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import org.joda.time.Interval;
import java.util.List;
/**
*/
@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)
})
public interface QuerySegmentSpec
{
public List<Interval> getIntervals();
public <T> QueryRunner<T> lookup(Query<T> query, QuerySegmentWalker walker);
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -29,7 +29,6 @@ import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.NoneShardSpec;
import com.metamx.druid.shard.SingleDimensionShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -20,6 +20,7 @@
package com.metamx.druid.query.extraction; package com.metamx.druid.query.extraction;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.query.spec.DimExtractionFn;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -20,6 +20,7 @@
package com.metamx.druid.query.extraction; package com.metamx.druid.query.extraction;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.query.spec.DimExtractionFn;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.metamx.druid.query.search.FragmentSearchQuerySpec; import com.metamx.druid.query.search.FragmentSearchQuerySpec;
import com.metamx.druid.query.search.SearchQuerySpec; import com.metamx.druid.query.search.SearchQuerySpec;
import io.druid.query.spec.DimExtractionFn;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -20,6 +20,7 @@
package com.metamx.druid.query.extraction; package com.metamx.druid.query.extraction;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import io.druid.query.spec.DimExtractionFn;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -28,14 +28,14 @@ import com.fasterxml.jackson.datatype.guava.GuavaModule;
/** /**
*/ */
public class DefaultObjectMapper extends ObjectMapper public class CommonObjectMapper extends ObjectMapper
{ {
public DefaultObjectMapper() public CommonObjectMapper()
{ {
this(null); this(null);
} }
public DefaultObjectMapper(JsonFactory factory) public CommonObjectMapper(JsonFactory factory)
{ {
super(factory); super(factory);
registerModule(new DruidDefaultSerializersModule()); registerModule(new DruidDefaultSerializersModule());

View File

@ -42,13 +42,13 @@ public class JacksonModule implements Module
@Provides @LazySingleton @Json @Provides @LazySingleton @Json
public ObjectMapper jsonMapper() public ObjectMapper jsonMapper()
{ {
return new DefaultObjectMapper(); return new CommonObjectMapper();
} }
@Provides @LazySingleton @Smile @Provides @LazySingleton @Smile
public ObjectMapper smileMapper() public ObjectMapper smileMapper()
{ {
ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); ObjectMapper retVal = new CommonObjectMapper(new SmileFactory());
retVal.getJsonFactory().setCodec(retVal); retVal.getJsonFactory().setCodec(retVal);
return retVal; return retVal;
} }

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,7 +22,7 @@ package com.metamx.druid;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.CommonObjectMapper;
import io.druid.granularity.DurationGranularity; import io.druid.granularity.DurationGranularity;
import io.druid.granularity.PeriodGranularity; import io.druid.granularity.PeriodGranularity;
import io.druid.granularity.QueryGranularity; import io.druid.granularity.QueryGranularity;
@ -437,7 +437,7 @@ public class QueryGranularityTest
@Test @Test
public void testSerializePeriod() throws Exception public void testSerializePeriod() throws Exception
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
String json = "{ \"type\": \"period\", \"period\": \"P1D\" }"; String json = "{ \"type\": \"period\", \"period\": \"P1D\" }";
QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); QueryGranularity gran = mapper.readValue(json, QueryGranularity.class);
@ -462,7 +462,7 @@ public class QueryGranularityTest
@Test @Test
public void testSerializeDuration() throws Exception public void testSerializeDuration() throws Exception
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
String json = "{ \"type\": \"duration\", \"duration\": \"3600000\" }"; String json = "{ \"type\": \"duration\", \"duration\": \"3600000\" }";
QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); QueryGranularity gran = mapper.readValue(json, QueryGranularity.class);
@ -479,7 +479,7 @@ public class QueryGranularityTest
@Test @Test
public void testSerializeSimple() throws Exception public void testSerializeSimple() throws Exception
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
Assert.assertEquals( Assert.assertEquals(
QueryGranularity.ALL, QueryGranularity.ALL,
@ -501,7 +501,7 @@ public class QueryGranularityTest
@Test @Test
public void testDeserializeSimple() throws Exception public void testDeserializeSimple() throws Exception
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"all\"", QueryGranularity.class)); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"all\"", QueryGranularity.class));
Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class)); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class));

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -22,8 +22,6 @@ package com.metamx.druid.histogram;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.druid.aggregation.Histogram; import com.metamx.druid.aggregation.Histogram;
import com.metamx.druid.jackson.DefaultObjectMapper;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -63,7 +61,7 @@ public class HistogramTest
Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0}; Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0};
Double[] visualCounts = { 123., 4., 56., 7. }; Double[] visualCounts = { 123., 4., 56., 7. };
ObjectMapper objectMapper = new DefaultObjectMapper(); ObjectMapper objectMapper = new ObjectMapper();
String json = objectMapper.writeValueAsString(h.asVisual()); String json = objectMapper.writeValueAsString(h.asVisual());
Map<String,Object> expectedObj = Maps.newLinkedHashMap(); Map<String,Object> expectedObj = Maps.newLinkedHashMap();

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -26,9 +26,9 @@ import org.junit.Test;
/** /**
*/ */
public class DefaultObjectMapperTest public class CommonObjectMapperTest
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
@Test @Test
public void testDateTime() throws Exception public void testDateTime() throws Exception

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -52,7 +52,7 @@ import com.metamx.druid.index.serde.FloatGenericColumnSupplier;
import com.metamx.druid.index.serde.LongGenericColumnPartSerde; import com.metamx.druid.index.serde.LongGenericColumnPartSerde;
import com.metamx.druid.index.serde.LongGenericColumnSupplier; import com.metamx.druid.index.serde.LongGenericColumnSupplier;
import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier; import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.CommonObjectMapper;
import com.metamx.druid.kv.ArrayIndexed; import com.metamx.druid.kv.ArrayIndexed;
import com.metamx.druid.kv.ByteBufferSerializer; import com.metamx.druid.kv.ByteBufferSerializer;
import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.ConciseCompressedIndexedInts;
@ -110,7 +110,7 @@ public class IndexIO
public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
// This should really be provided by DI, should be changed once we switch around to using a DI framework // This should really be provided by DI, should be changed once we switch around to using a DI framework
private static final ObjectMapper mapper = new DefaultObjectMapper(); private static final ObjectMapper mapper = new CommonObjectMapper();
private static volatile IndexIOHandler handler = null; private static volatile IndexIOHandler handler = null;
public static final int CURRENT_VERSION_ID = V9_VERSION; public static final int CURRENT_VERSION_ID = V9_VERSION;
@ -790,7 +790,7 @@ public class IndexIO
Map<String, Column> columns = Maps.newHashMap(); Map<String, Column> columns = Maps.newHashMap();
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new CommonObjectMapper();
for (String columnName : cols) { for (String columnName : cols) {
columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName)));

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 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.indexer.data; package com.metamx.druid.indexer.data;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
@ -6,7 +25,7 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.index.v1.SpatialDimensionSchema;
import com.metamx.druid.input.InputRow; import com.metamx.druid.input.InputRow;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.CommonObjectMapper;
import junit.framework.Assert; import junit.framework.Assert;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.junit.Test; import org.junit.Test;
@ -15,7 +34,7 @@ import java.nio.ByteBuffer;
public class InputRowParserSerdeTest public class InputRowParserSerdeTest
{ {
private final ObjectMapper jsonMapper = new DefaultObjectMapper(); private final ObjectMapper jsonMapper = new CommonObjectMapper();
@Test @Test
public void testStringInputRowParserSerde() throws Exception public void testStringInputRowParserSerde() throws Exception

View File

@ -1,87 +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.indexer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.MapUtils;
import com.metamx.common.logger.Logger;
import com.metamx.druid.jackson.DefaultObjectMapper;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
import java.util.Properties;
/**
*/
public class HadoopDruidIndexerAzkWrapper
{
private static final Logger log = new Logger(HadoopDruidIndexerAzkWrapper.class);
private static final String PROPERTY_PREFIX = "druid.indexer.";
private final String jobName;
private final Properties properties;
public HadoopDruidIndexerAzkWrapper(
String jobName,
Properties properties
)
{
this.jobName = jobName;
this.properties = properties;
}
public void run() throws Exception
{
final DefaultObjectMapper jsonMapper = new DefaultObjectMapper();
final List<Interval> dataInterval;
final Map<String, Object> theMap = Maps.newTreeMap();
for (String propertyName : properties.stringPropertyNames()) {
if (propertyName.startsWith(PROPERTY_PREFIX)) {
final String propValue = properties.getProperty(propertyName);
if (propValue.trim().startsWith("{") || propValue.trim().startsWith("[")) {
theMap.put(propertyName.substring(PROPERTY_PREFIX.length()), jsonMapper.readValue(propValue, Object.class));
}
else {
theMap.put(propertyName.substring(PROPERTY_PREFIX.length()), propValue);
}
}
}
log.info("Running with properties:");
for (Map.Entry<String, Object> entry : theMap.entrySet()) {
log.info("%30s => %s", entry.getKey(), entry.getValue());
}
dataInterval = Lists.transform(
Lists.newArrayList(MapUtils.getString(theMap, "timeInterval").split(",")), new StringIntervalFunction()
);
final HadoopDruidIndexerConfig config = jsonMapper.convertValue(theMap, HadoopDruidIndexerConfig.class);
config.setIntervals(dataInterval);
config.setVersion(new DateTime().toString());
new HadoopDruidIndexerJob(config).run();
}
}

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -36,10 +36,10 @@ import com.metamx.druid.indexing.common.TaskToolbox;
import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.TaskToolboxFactory;
import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.task.Task;
import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.joda.time.Interval; import org.joda.time.Interval;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -28,8 +28,8 @@ import com.metamx.druid.indexing.common.TaskStatus;
import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.task.Task;
import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunner;
import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.indexing.worker.config.WorkerConfig;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import io.druid.query.QuerySegmentWalker;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -31,7 +31,6 @@ import com.metamx.druid.input.InputRow;
import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.FinalizeResultsQueryRunner;
import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.Firehose;
import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.Plumber;
import com.metamx.druid.realtime.plumber.Sink; import com.metamx.druid.realtime.plumber.Sink;
@ -39,6 +38,7 @@ import com.metamx.emitter.EmittingLogger;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactory;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import org.joda.time.DateTime; import org.joda.time.DateTime;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -41,7 +41,6 @@ import com.metamx.druid.query.MetricsEmittingQueryRunner;
import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.NoopQueryRunner;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner; import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner;
import com.metamx.druid.query.segment.QuerySegmentWalker;
import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; import com.metamx.druid.query.segment.SpecificSegmentQueryRunner;
import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.druid.query.segment.SpecificSegmentSpec;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
@ -50,6 +49,7 @@ import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactory;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChest;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.QuerySegmentSpec;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -20,12 +20,12 @@
package com.metamx.druid.index.brita; package com.metamx.druid.index.brita;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.druid.query.extraction.DimExtractionFn;
import io.druid.data.Indexed; import io.druid.data.Indexed;
import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter; import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory; import io.druid.query.filter.ValueMatcherFactory;
import io.druid.query.spec.DimExtractionFn;
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
import java.util.List; import java.util.List;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -42,9 +42,9 @@ import com.metamx.druid.guice.annotations.Global;
import com.metamx.druid.index.brita.Filters; import com.metamx.druid.index.brita.Filters;
import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.MapBasedRow;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.dimension.DimensionSpec;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.spec.DimensionSpec;
import io.druid.segment.Cursor; import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelector;
import io.druid.segment.StorageAdapter; import io.druid.segment.StorageAdapter;

View File

@ -1,6 +1,6 @@
/* /*
* Druid - a distributed column store. * Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc. * Copyright (C) 2012, 2013 Metamarkets Group Inc.
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -38,7 +38,6 @@ import com.metamx.druid.input.MapBasedRow;
import com.metamx.druid.input.Row; import com.metamx.druid.input.Row;
import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.QueryRunnerTestHelper;
import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DefaultDimensionSpec;
import com.metamx.druid.query.dimension.DimensionSpec;
import com.metamx.druid.query.filter.RegexDimFilter; import com.metamx.druid.query.filter.RegexDimFilter;
import com.metamx.druid.query.group.having.EqualToHavingSpec; import com.metamx.druid.query.group.having.EqualToHavingSpec;
import com.metamx.druid.query.group.having.GreaterThanHavingSpec; import com.metamx.druid.query.group.having.GreaterThanHavingSpec;
@ -51,6 +50,7 @@ import io.druid.granularity.QueryGranularity;
import io.druid.query.Query; import io.druid.query.Query;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.spec.DimensionSpec;
import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.MultipleIntervalSegmentSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
@ -731,7 +731,7 @@ public class GroupByQueryRunnerTest
); );
TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal");
QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); QueryRunner<Row> mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner);
TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit");
} }