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 new file mode 100644 index 00000000000..e72b85a4423 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -0,0 +1,78 @@ +/* + * 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.metadata; + +import com.metamx.druid.BaseQuery; +import com.metamx.druid.Query; +import com.metamx.druid.query.segment.QuerySegmentSpec; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.SegmentMetadataResultValue; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.util.Map; + +public class SegmentMetadataQuery extends BaseQuery> +{ + + public SegmentMetadataQuery( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("context") Map context + ) + { + super( + dataSource, + querySegmentSpec, + context + ); + } + + @Override + public boolean hasFilters() + { + return false; + } + + @Override + public String getType() + { + return "segmentMetadata"; + } + + @Override + public Query> withOverriddenContext(Map contextOverride) + { + return new SegmentMetadataQuery( + getDataSource(), + getQuerySegmentSpec(), + computeOverridenContext(contextOverride) + ); + } + + @Override + public Query> withQuerySegmentSpec(QuerySegmentSpec spec) + { + return new SegmentMetadataQuery( + getDataSource(), + spec, + getContext() + ); + } +} 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 new file mode 100644 index 00000000000..14ef61b8d3c --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -0,0 +1,106 @@ +/* + * 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.metadata; + +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.metamx.common.guava.ConcatSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.query.CacheStrategy; +import com.metamx.druid.query.ConcatQueryRunner; +import com.metamx.druid.query.MetricManipulationFn; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.SegmentMetadataResultValue; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.codehaus.jackson.type.TypeReference; +import org.joda.time.Interval; +import org.joda.time.Minutes; + + +public class SegmentMetadataQueryQueryToolChest implements QueryToolChest, SegmentMetadataQuery> +{ + + private static final TypeReference> TYPE_REFERENCE = new TypeReference>(){}; + + @Override + public QueryRunner> mergeResults(final QueryRunner> runner) + { + return new ConcatQueryRunner>(Sequences.simple(ImmutableList.of(runner))); + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(SegmentMetadataQuery query) + { + int numMinutes = 0; + for (Interval interval : query.getIntervals()) { + numMinutes += Minutes.minutesIn(interval).getMinutes(); + } + + return new ServiceMetricEvent.Builder() + .setUser2(query.getDataSource()) + .setUser4(query.getType()) + .setUser5(Joiner.on(",").join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser9(Minutes.minutes(numMinutes).toString()); + } + + @Override + public Sequence> mergeSequences(Sequence>> seqOfSequences) + { + return new ConcatSequence>(seqOfSequences); + } + + @Override + public Function, Result> makeMetricManipulatorFn( + SegmentMetadataQuery query, MetricManipulationFn fn + ) + { + return Functions.identity(); + } + + @Override + public TypeReference> getResultTypeReference() + { + return TYPE_REFERENCE; + } + + @Override + public CacheStrategy, SegmentMetadataQuery> getCacheStrategy(SegmentMetadataQuery 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/result/SegmentMetadataResultValue.java b/client/src/main/java/com/metamx/druid/result/SegmentMetadataResultValue.java new file mode 100644 index 00000000000..5904264017e --- /dev/null +++ b/client/src/main/java/com/metamx/druid/result/SegmentMetadataResultValue.java @@ -0,0 +1,101 @@ +/* + * 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.result; + +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.util.Map; + +public class SegmentMetadataResultValue +{ + public static class Dimension { + @JsonProperty public long size; + @JsonProperty public int cardinality; + + @JsonCreator + public Dimension( + @JsonProperty("size") long size, + @JsonProperty("cardinality") int cardinality + ) + { + this.size = size; + this.cardinality = cardinality; + } + } + public static class Metric { + @JsonProperty public String type; + @JsonProperty public long size; + + @JsonCreator + public Metric( + @JsonProperty("type") String type, + @JsonProperty("size") long size + ) + { + this.type = type; + this.size = size; + } + } + + private final String id; + private final Map dimensions; + private final Map metrics; + private final long size; + + @JsonCreator + public SegmentMetadataResultValue( + @JsonProperty("id") String id, + @JsonProperty("dimensions") Map dimensions, + @JsonProperty("metrics") Map metrics, + @JsonProperty("size") long size + + ) + { + this.id = id; + this.dimensions = dimensions; + this.metrics = metrics; + this.size = size; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public Map getDimensions() + { + return dimensions; + } + + @JsonProperty + public Map getMetrics() + { + return metrics; + } + + @JsonProperty + public long getSize() + { + return size; + } +} diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java new file mode 100644 index 00000000000..edc4ea3eda8 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryEngine.java @@ -0,0 +1,109 @@ +/* + * 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.metadata; + +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.metamx.common.IAE; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.SimpleSequence; +import com.metamx.druid.BaseStorageAdapter; +import com.metamx.druid.StorageAdapter; +import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; +import com.metamx.druid.kv.Indexed; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.SegmentMetadataResultValue; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.List; + + + +public class SegmentMetadataQueryEngine +{ + public Sequence> process( + final SegmentMetadataQuery query, + StorageAdapter storageAdapter + ) + { + final List intervals = query.getQuerySegmentSpec().getIntervals(); + if (intervals.size() != 1) { + throw new IAE("Should only have one interval, got[%s]", intervals); + } + + if(!(storageAdapter instanceof SegmentIdAttachedStorageAdapter) || + !(((SegmentIdAttachedStorageAdapter)storageAdapter).getDelegate() instanceof BaseStorageAdapter)) { + return Sequences.empty(); + } + + final BaseStorageAdapter adapter = (BaseStorageAdapter) + ((SegmentIdAttachedStorageAdapter) storageAdapter).getDelegate(); + + Function sizeDimension = new Function() + { + @Override + public SegmentMetadataResultValue.Dimension apply(@Nullable String input) + { + long size = 0; + final Indexed lookup = adapter.getDimValueLookup(input); + for (String dimVal : lookup) { + ImmutableConciseSet index = adapter.getInvertedIndex(input, dimVal); + size += index.size() * Charsets.UTF_8.encode(dimVal).capacity(); + } + return new SegmentMetadataResultValue.Dimension( + size, + adapter.getDimensionCardinality(input) + ); + } + }; + + // TODO: add metric storage size + + long totalSize = 0; + + HashMap dimensions = Maps.newHashMap(); + for(String input : adapter.getAvailableDimensions()) { + SegmentMetadataResultValue.Dimension d = sizeDimension.apply(input); + dimensions.put(input, d); + totalSize += d.size; + } + + return new SimpleSequence>( + ImmutableList.of( + new Result( + adapter.getMinTime(), + new SegmentMetadataResultValue( + storageAdapter.getSegmentIdentifier(), + dimensions, + ImmutableMap.of(), + totalSize + ) + ) + ) + ); + } +} \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java new file mode 100644 index 00000000000..ce624a944f4 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -0,0 +1,128 @@ +/* + * 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.metadata; + +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.metamx.common.ISE; +import com.metamx.common.guava.ExecutorExecutingSequence; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.Query; +import com.metamx.druid.StorageAdapter; +import com.metamx.druid.query.ConcatQueryRunner; +import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.QueryRunnerFactory; +import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQueryEngine; +import com.metamx.druid.query.metadata.SegmentMetadataQueryQueryToolChest; +import com.metamx.druid.result.SegmentMetadataResultValue; +import com.metamx.druid.result.Result; + +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; + +public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory, SegmentMetadataQuery> +{ + private static final SegmentMetadataQueryQueryToolChest toolChest = new SegmentMetadataQueryQueryToolChest() + { + @Override + public QueryRunner> mergeResults(QueryRunner> runner) + { + return new ConcatQueryRunner>(Sequences.simple(ImmutableList.of(runner))); + } + }; + + + @Override + public QueryRunner> createRunner(final StorageAdapter adapter) + { + return new QueryRunner>() + { + @Override + public Sequence> run(Query> query) + { + if (!(query instanceof SegmentMetadataQuery)) { + throw new ISE("Got a [%s] which isn't a %s", query.getClass(), SegmentMetadataQuery.class); + } + return new SegmentMetadataQueryEngine().process((SegmentMetadataQuery) query, adapter); + } + }; + } + + @Override + public QueryRunner> mergeRunners( + final ExecutorService queryExecutor, Iterable>> queryRunners + ) + { + return new ConcatQueryRunner>( + Sequences.map( + Sequences.simple(queryRunners), + new Function>, QueryRunner>>() + { + @Override + public QueryRunner> apply(final QueryRunner> input) + { + return new QueryRunner>() + { + @Override + public Sequence> run(final Query> query) + { + + Future>> future = queryExecutor.submit( + new Callable>>() + { + @Override + public Sequence> call() throws Exception + { + return new ExecutorExecutingSequence>( + input.run(query), + queryExecutor + ); + } + } + ); + try { + return future.get(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + }; + } + } + ) + ); + } + + @Override + public QueryToolChest getToolchest() + { + return toolChest; + } +}