mirror of https://github.com/apache/druid.git
merge master
This commit is contained in:
commit
e12da51057
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.segment.QuerySegmentSpec;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
@ -72,7 +73,12 @@ public abstract class BaseQuery<T> implements Query<T>
|
|||
@Override
|
||||
public Sequence<T> run(QuerySegmentWalker walker)
|
||||
{
|
||||
return querySegmentSpec.lookup(this, walker).run(this);
|
||||
return run(querySegmentSpec.lookup(this, walker));
|
||||
}
|
||||
|
||||
public Sequence<T> run(QueryRunner<T> runner)
|
||||
{
|
||||
return runner.run(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid;
|
||||
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.group.GroupByQuery;
|
||||
import com.metamx.druid.query.metadata.SegmentMetadataQuery;
|
||||
import com.metamx.druid.query.search.SearchQuery;
|
||||
|
@ -57,6 +58,8 @@ public interface Query<T>
|
|||
|
||||
public Sequence<T> run(QuerySegmentWalker walker);
|
||||
|
||||
public Sequence<T> run(QueryRunner<T> runner);
|
||||
|
||||
public List<Interval> getIntervals();
|
||||
|
||||
public Duration getDuration();
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.collect.Iterators;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
|
@ -41,7 +42,6 @@ import com.metamx.druid.TimelineObjectHolder;
|
|||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.cache.Cache;
|
||||
import com.metamx.druid.client.cache.CacheBroker;
|
||||
import com.metamx.druid.client.selector.ServerSelector;
|
||||
import com.metamx.druid.partition.PartitionChunk;
|
||||
import com.metamx.druid.query.CacheStrategy;
|
||||
|
@ -54,6 +54,7 @@ import com.metamx.druid.query.segment.SegmentDescriptor;
|
|||
import com.metamx.druid.result.BySegmentResultValueClass;
|
||||
import com.metamx.druid.result.Result;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -64,6 +65,7 @@ import java.util.Collections;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
|
@ -74,19 +76,19 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final ServerView serverView;
|
||||
private final CacheBroker cacheBroker;
|
||||
private final Cache cache;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public CachingClusteredClient(
|
||||
QueryToolChestWarehouse warehouse,
|
||||
ServerView serverView,
|
||||
CacheBroker cacheBroker,
|
||||
Cache cache,
|
||||
ObjectMapper objectMapper
|
||||
)
|
||||
{
|
||||
this.warehouse = warehouse;
|
||||
this.serverView = serverView;
|
||||
this.cacheBroker = cacheBroker;
|
||||
this.cache = cache;
|
||||
this.objectMapper = objectMapper;
|
||||
|
||||
serverView.registerSegmentCallback(
|
||||
|
@ -98,7 +100,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
@Override
|
||||
public ServerView.CallbackAction segmentRemoved(DruidServer server, DataSegment segment)
|
||||
{
|
||||
CachingClusteredClient.this.cacheBroker.provideCache(segment.getIdentifier()).close();
|
||||
CachingClusteredClient.this.cache.close(segment.getIdentifier());
|
||||
return ServerView.CallbackAction.CONTINUE;
|
||||
}
|
||||
}
|
||||
|
@ -109,9 +111,10 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
|
||||
final CacheStrategy<T, Query<T>> strategy = toolChest.getCacheStrategy(query);
|
||||
final CacheStrategy<T, Object, Query<T>> strategy = toolChest.getCacheStrategy(query);
|
||||
|
||||
final Map<DruidServer, List<SegmentDescriptor>> serverSegments = Maps.newTreeMap();
|
||||
|
||||
final Map<DruidServer, List<SegmentDescriptor>> segs = Maps.newTreeMap();
|
||||
final List<Pair<DateTime, byte[]>> cachedResults = Lists.newArrayList();
|
||||
final Map<String, CachePopulator> cachePopulatorMap = Maps.newHashMap();
|
||||
|
||||
|
@ -131,10 +134,8 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
return Sequences.empty();
|
||||
}
|
||||
|
||||
byte[] queryCacheKey = null;
|
||||
if (strategy != null) {
|
||||
queryCacheKey = strategy.computeCacheKey(query);
|
||||
}
|
||||
// build set of segments to query
|
||||
Set<Pair<ServerSelector, SegmentDescriptor>> segments = Sets.newLinkedHashSet();
|
||||
|
||||
for (Interval interval : rewrittenQuery.getIntervals()) {
|
||||
List<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline.lookup(interval);
|
||||
|
@ -146,55 +147,67 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
holder.getInterval(), holder.getVersion(), chunk.getChunkNumber()
|
||||
);
|
||||
|
||||
if (queryCacheKey == null) {
|
||||
final DruidServer server = selector.pick();
|
||||
List<SegmentDescriptor> descriptors = segs.get(server);
|
||||
|
||||
if (descriptors == null) {
|
||||
descriptors = Lists.newArrayList();
|
||||
segs.put(server, descriptors);
|
||||
}
|
||||
|
||||
descriptors.add(descriptor);
|
||||
}
|
||||
else {
|
||||
final Interval segmentQueryInterval = holder.getInterval();
|
||||
final byte[] versionBytes = descriptor.getVersion().getBytes();
|
||||
|
||||
final byte[] cacheKey = ByteBuffer
|
||||
.allocate(16 + versionBytes.length + 4 + queryCacheKey.length)
|
||||
.putLong(segmentQueryInterval.getStartMillis())
|
||||
.putLong(segmentQueryInterval.getEndMillis())
|
||||
.put(versionBytes)
|
||||
.putInt(descriptor.getPartitionNumber())
|
||||
.put(queryCacheKey)
|
||||
.array();
|
||||
final String segmentIdentifier = selector.getSegment().getIdentifier();
|
||||
final Cache cache = cacheBroker.provideCache(segmentIdentifier);
|
||||
final byte[] cachedValue = cache.get(cacheKey);
|
||||
|
||||
if (useCache && cachedValue != null) {
|
||||
cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue));
|
||||
} else {
|
||||
final DruidServer server = selector.pick();
|
||||
List<SegmentDescriptor> descriptors = segs.get(server);
|
||||
|
||||
if (descriptors == null) {
|
||||
descriptors = Lists.newArrayList();
|
||||
segs.put(server, descriptors);
|
||||
}
|
||||
|
||||
descriptors.add(descriptor);
|
||||
cachePopulatorMap.put(
|
||||
String.format("%s_%s", segmentIdentifier, segmentQueryInterval),
|
||||
new CachePopulator(cache, objectMapper, cacheKey)
|
||||
);
|
||||
}
|
||||
}
|
||||
segments.add(Pair.of(selector, descriptor));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final byte[] queryCacheKey;
|
||||
if(strategy != null) {
|
||||
queryCacheKey = strategy.computeCacheKey(query);
|
||||
} else {
|
||||
queryCacheKey = null;
|
||||
}
|
||||
|
||||
// Pull cached segments from cache and remove from set of segments to query
|
||||
if(useCache && queryCacheKey != null) {
|
||||
Map<Pair<ServerSelector, SegmentDescriptor>, Cache.NamedKey> cacheKeys = Maps.newHashMap();
|
||||
for(Pair<ServerSelector, SegmentDescriptor> e : segments) {
|
||||
cacheKeys.put(e, computeSegmentCacheKey(e.lhs.getSegment().getIdentifier(), e.rhs, queryCacheKey));
|
||||
}
|
||||
|
||||
Map<Cache.NamedKey, byte[]> cachedValues = cache.getBulk(cacheKeys.values());
|
||||
|
||||
for(Map.Entry<Pair<ServerSelector, SegmentDescriptor>, Cache.NamedKey> entry : cacheKeys.entrySet()) {
|
||||
Pair<ServerSelector, SegmentDescriptor> segment = entry.getKey();
|
||||
Cache.NamedKey segmentCacheKey = entry.getValue();
|
||||
|
||||
final ServerSelector selector = segment.lhs;
|
||||
final SegmentDescriptor descriptor = segment.rhs;
|
||||
final Interval segmentQueryInterval = descriptor.getInterval();
|
||||
|
||||
final byte[] cachedValue = cachedValues.get(segmentCacheKey);
|
||||
|
||||
if (cachedValue != null) {
|
||||
cachedResults.add(Pair.of(segmentQueryInterval.getStart(), cachedValue));
|
||||
|
||||
// remove cached segment from set of segments to query
|
||||
segments.remove(segment);
|
||||
}
|
||||
else {
|
||||
final String segmentIdentifier = selector.getSegment().getIdentifier();
|
||||
cachePopulatorMap.put(
|
||||
String.format("%s_%s", segmentIdentifier, segmentQueryInterval),
|
||||
new CachePopulator(cache, objectMapper, segmentCacheKey)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Compile list of all segments not pulled from cache
|
||||
for(Pair<ServerSelector, SegmentDescriptor> segment : segments) {
|
||||
final DruidServer server = segment.lhs.pick();
|
||||
List<SegmentDescriptor> descriptors = serverSegments.get(server);
|
||||
|
||||
if (descriptors == null) {
|
||||
descriptors = Lists.newArrayList();
|
||||
serverSegments.put(server, descriptors);
|
||||
}
|
||||
|
||||
descriptors.add(segment.rhs);
|
||||
}
|
||||
|
||||
|
||||
return new LazySequence<T>(
|
||||
new Supplier<Sequence<T>>()
|
||||
{
|
||||
|
@ -229,6 +242,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
}
|
||||
|
||||
final Function<Object, T> pullFromCacheFunction = strategy.pullFromCache();
|
||||
final TypeReference<Object> cacheObjectClazz = strategy.getCacheObjectClazz();
|
||||
for (Pair<DateTime, byte[]> cachedResultPair : cachedResults) {
|
||||
final byte[] cachedResult = cachedResultPair.rhs;
|
||||
Sequence<Object> cachedSequence = new BaseSequence<Object, Iterator<Object>>(
|
||||
|
@ -243,7 +257,8 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
}
|
||||
|
||||
return objectMapper.readValues(
|
||||
objectMapper.getJsonFactory().createJsonParser(cachedResult), Object.class
|
||||
objectMapper.getJsonFactory().createJsonParser(cachedResult),
|
||||
cacheObjectClazz
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
@ -264,7 +279,7 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
@SuppressWarnings("unchecked")
|
||||
private void addSequencesFromServer(ArrayList<Pair<DateTime, Sequence<T>>> listOfSequences)
|
||||
{
|
||||
for (Map.Entry<DruidServer, List<SegmentDescriptor>> entry : segs.entrySet()) {
|
||||
for (Map.Entry<DruidServer, List<SegmentDescriptor>> entry : serverSegments.entrySet()) {
|
||||
final DruidServer server = entry.getKey();
|
||||
final List<SegmentDescriptor> descriptors = entry.getValue();
|
||||
|
||||
|
@ -328,13 +343,29 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
|
|||
);
|
||||
}
|
||||
|
||||
private Cache.NamedKey computeSegmentCacheKey(String segmentIdentifier, SegmentDescriptor descriptor, byte[] queryCacheKey)
|
||||
{
|
||||
final Interval segmentQueryInterval = descriptor.getInterval();
|
||||
final byte[] versionBytes = descriptor.getVersion().getBytes();
|
||||
|
||||
return new Cache.NamedKey(
|
||||
segmentIdentifier, ByteBuffer
|
||||
.allocate(16 + versionBytes.length + 4 + queryCacheKey.length)
|
||||
.putLong(segmentQueryInterval.getStartMillis())
|
||||
.putLong(segmentQueryInterval.getEndMillis())
|
||||
.put(versionBytes)
|
||||
.putInt(descriptor.getPartitionNumber())
|
||||
.put(queryCacheKey).array()
|
||||
);
|
||||
}
|
||||
|
||||
private static class CachePopulator
|
||||
{
|
||||
private final Cache cache;
|
||||
private final ObjectMapper mapper;
|
||||
private final byte[] key;
|
||||
private final Cache.NamedKey key;
|
||||
|
||||
public CachePopulator(Cache cache, ObjectMapper mapper, byte[] key)
|
||||
public CachePopulator(Cache cache, ObjectMapper mapper, Cache.NamedKey key)
|
||||
{
|
||||
this.cache = cache;
|
||||
this.mapper = mapper;
|
||||
|
|
|
@ -48,6 +48,8 @@ import java.util.Map;
|
|||
public class DataSegment implements Comparable<DataSegment>
|
||||
{
|
||||
public static String delimiter = "_";
|
||||
private final Integer binaryVersion;
|
||||
|
||||
public static String makeDataSegmentIdentifier(
|
||||
String dataSource,
|
||||
DateTime start,
|
||||
|
@ -89,6 +91,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
@JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> dimensions,
|
||||
@JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> metrics,
|
||||
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
||||
@JsonProperty("binaryVersion") Integer binaryVersion,
|
||||
@JsonProperty("size") long size
|
||||
)
|
||||
{
|
||||
|
@ -112,6 +115,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
? ImmutableList.<String>of()
|
||||
: ImmutableList.copyOf(Iterables.filter(metrics, nonEmpty));
|
||||
this.shardSpec = (shardSpec == null) ? new NoneShardSpec() : shardSpec;
|
||||
this.binaryVersion = binaryVersion;
|
||||
this.size = size;
|
||||
|
||||
this.identifier = makeDataSegmentIdentifier(
|
||||
|
@ -172,6 +176,12 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
return shardSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Integer getBinaryVersion()
|
||||
{
|
||||
return binaryVersion;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getSize()
|
||||
{
|
||||
|
@ -209,6 +219,11 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
return builder(this).version(version).build();
|
||||
}
|
||||
|
||||
public DataSegment withBinaryVersion(int binaryVersion)
|
||||
{
|
||||
return builder(this).binaryVersion(binaryVersion).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(DataSegment dataSegment)
|
||||
{
|
||||
|
@ -287,6 +302,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
private List<String> dimensions;
|
||||
private List<String> metrics;
|
||||
private ShardSpec shardSpec;
|
||||
private Integer binaryVersion;
|
||||
private long size;
|
||||
|
||||
public Builder()
|
||||
|
@ -307,6 +323,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
this.dimensions = segment.getDimensions();
|
||||
this.metrics = segment.getMetrics();
|
||||
this.shardSpec = segment.getShardSpec();
|
||||
this.binaryVersion = segment.getBinaryVersion();
|
||||
this.size = segment.getSize();
|
||||
}
|
||||
|
||||
|
@ -352,6 +369,12 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder binaryVersion(Integer binaryVersion)
|
||||
{
|
||||
this.binaryVersion = binaryVersion;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder size(long size)
|
||||
{
|
||||
this.size = size;
|
||||
|
@ -374,6 +397,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
dimensions,
|
||||
metrics,
|
||||
shardSpec,
|
||||
binaryVersion,
|
||||
size
|
||||
);
|
||||
}
|
||||
|
|
|
@ -19,13 +19,74 @@
|
|||
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.primitives.Ints;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* An interface to limit the operations that can be done on a Cache so that it is easier to reason about what
|
||||
* is actually going to be done.
|
||||
*/
|
||||
public interface Cache
|
||||
{
|
||||
public byte[] get(byte[] key);
|
||||
public void put(byte[] key, byte[] value);
|
||||
public void close();
|
||||
public byte[] get(NamedKey key);
|
||||
public void put(NamedKey key, byte[] value);
|
||||
public Map<NamedKey, byte[]> getBulk(Iterable<NamedKey> keys);
|
||||
|
||||
public void close(String namespace);
|
||||
|
||||
public CacheStats getStats();
|
||||
|
||||
public class NamedKey
|
||||
{
|
||||
final public String namespace;
|
||||
final public byte[] key;
|
||||
|
||||
public NamedKey(String namespace, byte[] key) {
|
||||
Preconditions.checkArgument(namespace != null, "namespace must not be null");
|
||||
Preconditions.checkArgument(key != null, "key must not be null");
|
||||
this.namespace = namespace;
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
public byte[] toByteArray() {
|
||||
final byte[] nsBytes = this.namespace.getBytes(Charsets.UTF_8);
|
||||
return ByteBuffer.allocate(Ints.BYTES + nsBytes.length + this.key.length)
|
||||
.putInt(nsBytes.length)
|
||||
.put(nsBytes)
|
||||
.put(this.key).array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
NamedKey namedKey = (NamedKey) o;
|
||||
|
||||
if (!namespace.equals(namedKey.namespace)) {
|
||||
return false;
|
||||
}
|
||||
if (!Arrays.equals(key, namedKey.key)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = namespace.hashCode();
|
||||
result = 31 * result + Arrays.hashCode(key);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,21 +27,21 @@ import com.metamx.metrics.AbstractMonitor;
|
|||
*/
|
||||
public class CacheMonitor extends AbstractMonitor
|
||||
{
|
||||
private final CacheBroker cacheBroker;
|
||||
private final Cache cache;
|
||||
|
||||
private volatile CacheStats prevCacheStats = null;
|
||||
|
||||
public CacheMonitor(
|
||||
CacheBroker cacheBroker
|
||||
Cache cache
|
||||
)
|
||||
{
|
||||
this.cacheBroker = cacheBroker;
|
||||
this.cache = cache;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
final CacheStats currCacheStats = cacheBroker.getStats();
|
||||
final CacheStats currCacheStats = cache.getStats();
|
||||
final CacheStats deltaCacheStats = currCacheStats.delta(prevCacheStats);
|
||||
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
|
|
|
@ -0,0 +1,158 @@
|
|||
/*
|
||||
* 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.client.cache;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MapCache implements Cache
|
||||
{
|
||||
private final Map<ByteBuffer, byte[]> baseMap;
|
||||
private final ByteCountingLRUMap byteCountingLRUMap;
|
||||
|
||||
private final Map<String, byte[]> namespaceId;
|
||||
private final AtomicInteger ids;
|
||||
|
||||
private final Object clearLock = new Object();
|
||||
|
||||
private final AtomicLong hitCount = new AtomicLong(0);
|
||||
private final AtomicLong missCount = new AtomicLong(0);
|
||||
|
||||
public static com.metamx.druid.client.cache.Cache create(final MapCacheConfig config)
|
||||
{
|
||||
return new MapCache(
|
||||
new ByteCountingLRUMap(
|
||||
config.getInitialSize(),
|
||||
config.getLogEvictionCount(),
|
||||
config.getSizeInBytes()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
MapCache(
|
||||
ByteCountingLRUMap byteCountingLRUMap
|
||||
)
|
||||
{
|
||||
this.byteCountingLRUMap = byteCountingLRUMap;
|
||||
|
||||
this.baseMap = Collections.synchronizedMap(byteCountingLRUMap);
|
||||
|
||||
namespaceId = Maps.newHashMap();
|
||||
ids = new AtomicInteger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheStats getStats()
|
||||
{
|
||||
return new CacheStats(
|
||||
hitCount.get(),
|
||||
missCount.get(),
|
||||
byteCountingLRUMap.size(),
|
||||
byteCountingLRUMap.getNumBytes(),
|
||||
byteCountingLRUMap.getEvictionCount(),
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] get(NamedKey key)
|
||||
{
|
||||
final byte[] retVal = baseMap.get(computeKey(getNamespaceId(key.namespace), key.key));
|
||||
if (retVal == null) {
|
||||
missCount.incrementAndGet();
|
||||
} else {
|
||||
hitCount.incrementAndGet();
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(NamedKey key, byte[] value)
|
||||
{
|
||||
synchronized (clearLock) {
|
||||
baseMap.put(computeKey(getNamespaceId(key.namespace), key.key), value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<NamedKey, byte[]> getBulk(Iterable<NamedKey> keys)
|
||||
{
|
||||
Map<NamedKey, byte[]> retVal = Maps.newHashMap();
|
||||
for(NamedKey key : keys) {
|
||||
retVal.put(key, get(key));
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(String namespace)
|
||||
{
|
||||
byte[] idBytes;
|
||||
synchronized (namespaceId) {
|
||||
idBytes = getNamespaceId(namespace);
|
||||
if(idBytes == null) return;
|
||||
|
||||
namespaceId.remove(namespace);
|
||||
}
|
||||
synchronized (clearLock) {
|
||||
Iterator<ByteBuffer> iter = baseMap.keySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
ByteBuffer next = iter.next();
|
||||
|
||||
if (next.get(0) == idBytes[0]
|
||||
&& next.get(1) == idBytes[1]
|
||||
&& next.get(2) == idBytes[2]
|
||||
&& next.get(3) == idBytes[3]) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] getNamespaceId(final String identifier)
|
||||
{
|
||||
synchronized (namespaceId) {
|
||||
byte[] idBytes = namespaceId.get(identifier);
|
||||
if (idBytes != null) {
|
||||
return idBytes;
|
||||
}
|
||||
|
||||
idBytes = Ints.toByteArray(ids.getAndIncrement());
|
||||
namespaceId.put(identifier, idBytes);
|
||||
return idBytes;
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer computeKey(byte[] idBytes, byte[] key)
|
||||
{
|
||||
final ByteBuffer retVal = ByteBuffer.allocate(key.length + 4).put(idBytes).put(key);
|
||||
retVal.rewind();
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -1,165 +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.client.cache;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.ISE;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class MapCacheBroker implements CacheBroker
|
||||
{
|
||||
private final Map<ByteBuffer, byte[]> baseMap;
|
||||
private final ByteCountingLRUMap byteCountingLRUMap;
|
||||
|
||||
private final Map<String, Cache> cacheCache;
|
||||
private final AtomicInteger ids;
|
||||
|
||||
private final Object clearLock = new Object();
|
||||
|
||||
private final AtomicLong hitCount = new AtomicLong(0);
|
||||
private final AtomicLong missCount = new AtomicLong(0);
|
||||
|
||||
public static CacheBroker create(final MapCacheBrokerConfig config)
|
||||
{
|
||||
return new MapCacheBroker(
|
||||
new ByteCountingLRUMap(
|
||||
config.getInitialSize(),
|
||||
config.getLogEvictionCount(),
|
||||
config.getSizeInBytes()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
MapCacheBroker(
|
||||
ByteCountingLRUMap byteCountingLRUMap
|
||||
)
|
||||
{
|
||||
this.byteCountingLRUMap = byteCountingLRUMap;
|
||||
|
||||
this.baseMap = Collections.synchronizedMap(byteCountingLRUMap);
|
||||
|
||||
cacheCache = Maps.newHashMap();
|
||||
ids = new AtomicInteger();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public CacheStats getStats()
|
||||
{
|
||||
return new CacheStats(
|
||||
hitCount.get(),
|
||||
missCount.get(),
|
||||
byteCountingLRUMap.size(),
|
||||
byteCountingLRUMap.getNumBytes(),
|
||||
byteCountingLRUMap.getEvictionCount(),
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cache provideCache(final String identifier)
|
||||
{
|
||||
synchronized (cacheCache) {
|
||||
final Cache cachedCache = cacheCache.get(identifier);
|
||||
if (cachedCache != null) {
|
||||
return cachedCache;
|
||||
}
|
||||
|
||||
final byte[] myIdBytes = Ints.toByteArray(ids.getAndIncrement());
|
||||
|
||||
final Cache theCache = new Cache()
|
||||
{
|
||||
volatile boolean open = true;
|
||||
|
||||
@Override
|
||||
public byte[] get(byte[] key)
|
||||
{
|
||||
if (open) {
|
||||
final byte[] retVal = baseMap.get(computeKey(key));
|
||||
if (retVal == null) {
|
||||
missCount.incrementAndGet();
|
||||
} else {
|
||||
hitCount.incrementAndGet();
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
throw new ISE("Cache for identifier[%s] is closed.", identifier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(byte[] key, byte[] value)
|
||||
{
|
||||
synchronized (clearLock) {
|
||||
if (open) {
|
||||
baseMap.put(computeKey(key), value);
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw new ISE("Cache for identifier[%s] is closed.", identifier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
synchronized (cacheCache) {
|
||||
cacheCache.remove(identifier);
|
||||
}
|
||||
synchronized (clearLock) {
|
||||
if (open) {
|
||||
open = false;
|
||||
|
||||
Iterator<ByteBuffer> iter = baseMap.keySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
ByteBuffer next = iter.next();
|
||||
|
||||
if (next.get(0) == myIdBytes[0]
|
||||
&& next.get(1) == myIdBytes[1]
|
||||
&& next.get(2) == myIdBytes[2]
|
||||
&& next.get(3) == myIdBytes[3]) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer computeKey(byte[] key)
|
||||
{
|
||||
final ByteBuffer retVal = ByteBuffer.allocate(key.length + 4).put(myIdBytes).put(key);
|
||||
retVal.rewind();
|
||||
return retVal;
|
||||
}
|
||||
};
|
||||
|
||||
cacheCache.put(identifier, theCache);
|
||||
|
||||
return theCache;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -24,7 +24,7 @@ import org.skife.config.Default;
|
|||
|
||||
/**
|
||||
*/
|
||||
public abstract class MapCacheBrokerConfig
|
||||
public abstract class MapCacheConfig
|
||||
{
|
||||
@Config("${prefix}.sizeInBytes")
|
||||
@Default("0")
|
|
@ -0,0 +1,236 @@
|
|||
/*
|
||||
* 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.client.cache;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import net.spy.memcached.AddrUtil;
|
||||
import net.spy.memcached.ConnectionFactoryBuilder;
|
||||
import net.spy.memcached.DefaultHashAlgorithm;
|
||||
import net.spy.memcached.FailureMode;
|
||||
import net.spy.memcached.MemcachedClient;
|
||||
import net.spy.memcached.MemcachedClientIF;
|
||||
import net.spy.memcached.internal.BulkFuture;
|
||||
import net.spy.memcached.transcoders.SerializingTranscoder;
|
||||
import org.apache.commons.codec.digest.DigestUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
public class MemcachedCache implements Cache
|
||||
{
|
||||
public static MemcachedCache create(final MemcachedCacheConfig config)
|
||||
{
|
||||
try {
|
||||
SerializingTranscoder transcoder = new SerializingTranscoder(config.getMaxObjectSize());
|
||||
// disable compression
|
||||
transcoder.setCompressionThreshold(Integer.MAX_VALUE);
|
||||
|
||||
return new MemcachedCache(
|
||||
new MemcachedClient(
|
||||
new ConnectionFactoryBuilder().setProtocol(ConnectionFactoryBuilder.Protocol.BINARY)
|
||||
.setHashAlg(DefaultHashAlgorithm.FNV1A_64_HASH)
|
||||
.setLocatorType(ConnectionFactoryBuilder.Locator.CONSISTENT)
|
||||
.setDaemon(true)
|
||||
.setFailureMode(FailureMode.Retry)
|
||||
.setTranscoder(transcoder)
|
||||
.setShouldOptimize(true)
|
||||
.build(),
|
||||
AddrUtil.getAddresses(config.getHosts())
|
||||
),
|
||||
config.getMemcachedPrefix(),
|
||||
config.getTimeout(),
|
||||
config.getExpiration()
|
||||
);
|
||||
} catch(IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
private final int timeout;
|
||||
private final int expiration;
|
||||
private final String memcachedPrefix;
|
||||
|
||||
private final MemcachedClientIF client;
|
||||
|
||||
private final AtomicLong hitCount = new AtomicLong(0);
|
||||
private final AtomicLong missCount = new AtomicLong(0);
|
||||
private final AtomicLong timeoutCount = new AtomicLong(0);
|
||||
|
||||
MemcachedCache(MemcachedClientIF client, String memcachedPrefix, int timeout, int expiration) {
|
||||
Preconditions.checkArgument(memcachedPrefix.length() <= MAX_PREFIX_LENGTH,
|
||||
"memcachedPrefix length [%d] exceeds maximum length [%d]",
|
||||
memcachedPrefix.length(),
|
||||
MAX_PREFIX_LENGTH);
|
||||
this.timeout = timeout;
|
||||
this.expiration = expiration;
|
||||
this.client = client;
|
||||
this.memcachedPrefix = memcachedPrefix;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheStats getStats()
|
||||
{
|
||||
return new CacheStats(
|
||||
hitCount.get(),
|
||||
missCount.get(),
|
||||
0,
|
||||
0,
|
||||
0,
|
||||
timeoutCount.get()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] get(NamedKey key)
|
||||
{
|
||||
Future<Object> future = client.asyncGet(computeKeyHash(memcachedPrefix, key));
|
||||
try {
|
||||
byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS);
|
||||
if(bytes != null) {
|
||||
hitCount.incrementAndGet();
|
||||
}
|
||||
else {
|
||||
missCount.incrementAndGet();
|
||||
}
|
||||
return bytes == null ? null : deserializeValue(key, bytes);
|
||||
}
|
||||
catch(TimeoutException e) {
|
||||
timeoutCount.incrementAndGet();
|
||||
future.cancel(false);
|
||||
return null;
|
||||
}
|
||||
catch(InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch(ExecutionException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(NamedKey key, byte[] value)
|
||||
{
|
||||
client.set(computeKeyHash(memcachedPrefix, key), expiration, serializeValue(key, value));
|
||||
}
|
||||
|
||||
private static byte[] serializeValue(NamedKey key, byte[] value) {
|
||||
byte[] keyBytes = key.toByteArray();
|
||||
return ByteBuffer.allocate(Ints.BYTES + keyBytes.length + value.length)
|
||||
.putInt(keyBytes.length)
|
||||
.put(keyBytes)
|
||||
.put(value)
|
||||
.array();
|
||||
}
|
||||
|
||||
private static byte[] deserializeValue(NamedKey key, byte[] bytes) {
|
||||
ByteBuffer buf = ByteBuffer.wrap(bytes);
|
||||
|
||||
final int keyLength = buf.getInt();
|
||||
byte[] keyBytes = new byte[keyLength];
|
||||
buf.get(keyBytes);
|
||||
byte[] value = new byte[buf.remaining()];
|
||||
buf.get(value);
|
||||
|
||||
Preconditions.checkState(Arrays.equals(keyBytes, key.toByteArray()),
|
||||
"Keys do not match, possible hash collision?");
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<NamedKey, byte[]> getBulk(Iterable<NamedKey> keys)
|
||||
{
|
||||
Map<String, NamedKey> keyLookup = Maps.uniqueIndex(
|
||||
keys,
|
||||
new Function<NamedKey, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(
|
||||
@Nullable NamedKey input
|
||||
)
|
||||
{
|
||||
return computeKeyHash(memcachedPrefix, input);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
BulkFuture<Map<String, Object>> future = client.asyncGetBulk(keyLookup.keySet());
|
||||
|
||||
try {
|
||||
Map<String, Object> some = future.getSome(timeout, TimeUnit.MILLISECONDS);
|
||||
|
||||
if(future.isTimeout()) {
|
||||
future.cancel(false);
|
||||
timeoutCount.incrementAndGet();
|
||||
}
|
||||
missCount.addAndGet(keyLookup.size() - some.size());
|
||||
hitCount.addAndGet(some.size());
|
||||
|
||||
Map<NamedKey, byte[]> results = Maps.newHashMap();
|
||||
for(Map.Entry<String, Object> entry : some.entrySet()) {
|
||||
final NamedKey key = keyLookup.get(entry.getKey());
|
||||
final byte[] value = (byte[]) entry.getValue();
|
||||
results.put(
|
||||
key,
|
||||
value == null ? null : deserializeValue(key, value)
|
||||
);
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
catch(InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch(ExecutionException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(String namespace)
|
||||
{
|
||||
// no resources to cleanup
|
||||
}
|
||||
|
||||
public static final int MAX_PREFIX_LENGTH =
|
||||
MemcachedClientIF.MAX_KEY_LENGTH
|
||||
- 40 // length of namespace hash
|
||||
- 40 // length of key hash
|
||||
- 2 // length of separators
|
||||
;
|
||||
|
||||
private static String computeKeyHash(String memcachedPrefix, NamedKey key) {
|
||||
// hash keys to keep things under 250 characters for memcached
|
||||
return memcachedPrefix + ":" + DigestUtils.sha1Hex(key.namespace) + ":" + DigestUtils.sha1Hex(key.key);
|
||||
}
|
||||
}
|
|
@ -1,145 +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.client.cache;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import net.iharder.base64.Base64;
|
||||
import net.spy.memcached.AddrUtil;
|
||||
import net.spy.memcached.ConnectionFactoryBuilder;
|
||||
import net.spy.memcached.DefaultHashAlgorithm;
|
||||
import net.spy.memcached.FailureMode;
|
||||
import net.spy.memcached.MemcachedClient;
|
||||
import net.spy.memcached.MemcachedClientIF;
|
||||
import net.spy.memcached.transcoders.SerializingTranscoder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
public class MemcachedCacheBroker implements CacheBroker
|
||||
{
|
||||
public static MemcachedCacheBroker create(final MemcachedCacheBrokerConfig config)
|
||||
{
|
||||
try {
|
||||
SerializingTranscoder transcoder = new SerializingTranscoder(config.getMaxObjectSize());
|
||||
// disable compression
|
||||
transcoder.setCompressionThreshold(Integer.MAX_VALUE);
|
||||
|
||||
return new MemcachedCacheBroker(
|
||||
new MemcachedClient(
|
||||
new ConnectionFactoryBuilder().setProtocol(ConnectionFactoryBuilder.Protocol.BINARY)
|
||||
.setHashAlg(DefaultHashAlgorithm.FNV1A_64_HASH)
|
||||
.setLocatorType(ConnectionFactoryBuilder.Locator.CONSISTENT)
|
||||
.setDaemon(true)
|
||||
.setFailureMode(FailureMode.Retry)
|
||||
.setTranscoder(transcoder)
|
||||
.setShouldOptimize(true)
|
||||
.build(),
|
||||
AddrUtil.getAddresses(config.getHosts())
|
||||
),
|
||||
config.getTimeout(),
|
||||
config.getExpiration()
|
||||
);
|
||||
} catch(IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
private final int timeout;
|
||||
private final int expiration;
|
||||
|
||||
private final MemcachedClientIF client;
|
||||
|
||||
private final AtomicLong hitCount = new AtomicLong(0);
|
||||
private final AtomicLong missCount = new AtomicLong(0);
|
||||
private final AtomicLong timeoutCount = new AtomicLong(0);
|
||||
|
||||
MemcachedCacheBroker(MemcachedClientIF client, int timeout, int expiration) {
|
||||
this.timeout = timeout;
|
||||
this.expiration = expiration;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheStats getStats()
|
||||
{
|
||||
return new CacheStats(
|
||||
hitCount.get(),
|
||||
missCount.get(),
|
||||
0,
|
||||
0,
|
||||
0,
|
||||
timeoutCount.get()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cache provideCache(final String identifier)
|
||||
{
|
||||
return new Cache()
|
||||
{
|
||||
@Override
|
||||
public byte[] get(byte[] key)
|
||||
{
|
||||
Future<Object> future = client.asyncGet(computeKey(identifier, key));
|
||||
try {
|
||||
byte[] bytes = (byte[]) future.get(timeout, TimeUnit.MILLISECONDS);
|
||||
if(bytes != null) {
|
||||
hitCount.incrementAndGet();
|
||||
}
|
||||
else {
|
||||
missCount.incrementAndGet();
|
||||
}
|
||||
return bytes;
|
||||
}
|
||||
catch(TimeoutException e) {
|
||||
timeoutCount.incrementAndGet();
|
||||
future.cancel(false);
|
||||
return null;
|
||||
}
|
||||
catch(InterruptedException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
catch(ExecutionException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(byte[] key, byte[] value)
|
||||
{
|
||||
client.set(computeKey(identifier, key), expiration, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
// no resources to cleanup
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private String computeKey(String identifier, byte[] key) {
|
||||
return identifier + Base64.encodeBytes(key, Base64.DONT_BREAK_LINES);
|
||||
}
|
||||
}
|
|
@ -3,10 +3,10 @@ package com.metamx.druid.client.cache;
|
|||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
public abstract class MemcachedCacheBrokerConfig
|
||||
public abstract class MemcachedCacheConfig
|
||||
{
|
||||
@Config("${prefix}.expiration")
|
||||
@Default("31536000")
|
||||
@Default("2592000")
|
||||
public abstract int getExpiration();
|
||||
|
||||
@Config("${prefix}.timeout")
|
||||
|
@ -17,5 +17,10 @@ public abstract class MemcachedCacheBrokerConfig
|
|||
public abstract String getHosts();
|
||||
|
||||
@Config("${prefix}.maxObjectSize")
|
||||
@Default("52428800")
|
||||
public abstract int getMaxObjectSize();
|
||||
|
||||
@Config("${prefix}.memcachedPrefix")
|
||||
@Default("druid")
|
||||
public abstract String getMemcachedPrefix();
|
||||
}
|
|
@ -34,13 +34,13 @@ import com.metamx.druid.client.BrokerServerView;
|
|||
import com.metamx.druid.client.CachingClusteredClient;
|
||||
import com.metamx.druid.client.ClientConfig;
|
||||
import com.metamx.druid.client.ClientInventoryManager;
|
||||
import com.metamx.druid.client.cache.CacheBroker;
|
||||
import com.metamx.druid.client.cache.Cache;
|
||||
import com.metamx.druid.client.cache.CacheConfig;
|
||||
import com.metamx.druid.client.cache.CacheMonitor;
|
||||
import com.metamx.druid.client.cache.MapCacheBroker;
|
||||
import com.metamx.druid.client.cache.MapCacheBrokerConfig;
|
||||
import com.metamx.druid.client.cache.MemcachedCacheBroker;
|
||||
import com.metamx.druid.client.cache.MemcachedCacheBrokerConfig;
|
||||
import com.metamx.druid.client.cache.MapCache;
|
||||
import com.metamx.druid.client.cache.MapCacheConfig;
|
||||
import com.metamx.druid.client.cache.MemcachedCache;
|
||||
import com.metamx.druid.client.cache.MemcachedCacheConfig;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -78,7 +78,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
|
||||
private QueryToolChestWarehouse warehouse = null;
|
||||
private HttpClient brokerHttpClient = null;
|
||||
private CacheBroker cacheBroker = null;
|
||||
private Cache cache = null;
|
||||
|
||||
private boolean useDiscovery = true;
|
||||
|
||||
|
@ -122,15 +122,15 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
return this;
|
||||
}
|
||||
|
||||
public CacheBroker getCacheBroker()
|
||||
public Cache getCache()
|
||||
{
|
||||
initializeCacheBroker();
|
||||
return cacheBroker;
|
||||
return cache;
|
||||
}
|
||||
|
||||
public BrokerNode setCacheBroker(CacheBroker cacheBroker)
|
||||
public BrokerNode setCache(Cache cache)
|
||||
{
|
||||
checkFieldNotSetAndSet("cacheBroker", cacheBroker);
|
||||
checkFieldNotSetAndSet("cache", cache);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -185,7 +185,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
final Lifecycle lifecycle = getLifecycle();
|
||||
|
||||
final List<Monitor> monitors = getMonitors();
|
||||
monitors.add(new CacheMonitor(cacheBroker));
|
||||
monitors.add(new CacheMonitor(cache));
|
||||
startMonitoring(monitors);
|
||||
|
||||
final BrokerServerView view = new BrokerServerView(warehouse, getSmileMapper(), brokerHttpClient);
|
||||
|
@ -194,7 +194,7 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
);
|
||||
lifecycle.addManagedInstance(clientInventoryManager);
|
||||
|
||||
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cacheBroker, getSmileMapper());
|
||||
final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper());
|
||||
lifecycle.addManagedInstance(baseClient);
|
||||
|
||||
|
||||
|
@ -239,25 +239,25 @@ public class BrokerNode extends QueryableNode<BrokerNode>
|
|||
|
||||
private void initializeCacheBroker()
|
||||
{
|
||||
if (cacheBroker == null) {
|
||||
if (cache == null) {
|
||||
String cacheType = getConfigFactory()
|
||||
.build(CacheConfig.class)
|
||||
.getType();
|
||||
|
||||
if (cacheType.equals(CACHE_TYPE_LOCAL)) {
|
||||
setCacheBroker(
|
||||
MapCacheBroker.create(
|
||||
setCache(
|
||||
MapCache.create(
|
||||
getConfigFactory().buildWithReplacements(
|
||||
MapCacheBrokerConfig.class,
|
||||
MapCacheConfig.class,
|
||||
ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX)
|
||||
)
|
||||
)
|
||||
);
|
||||
} else if (cacheType.equals(CACHE_TYPE_MEMCACHED)) {
|
||||
setCacheBroker(
|
||||
MemcachedCacheBroker.create(
|
||||
setCache(
|
||||
MemcachedCache.create(
|
||||
getConfigFactory().buildWithReplacements(
|
||||
MemcachedCacheBrokerConfig.class,
|
||||
MemcachedCacheConfig.class,
|
||||
ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* 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.http;
|
||||
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.client.DirectDruidClient;
|
||||
import com.metamx.druid.query.FinalizeResultsQueryRunner;
|
||||
import com.metamx.druid.query.QueryRunner;
|
||||
import com.metamx.druid.query.QueryToolChestWarehouse;
|
||||
import com.metamx.druid.query.segment.QuerySegmentWalker;
|
||||
import com.metamx.druid.query.segment.SegmentDescriptor;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DirectClientQuerySegmentWalker implements QuerySegmentWalker
|
||||
{
|
||||
private final QueryToolChestWarehouse warehouse;
|
||||
private final DirectDruidClient baseClient;
|
||||
|
||||
public DirectClientQuerySegmentWalker(
|
||||
QueryToolChestWarehouse warehouse,
|
||||
DirectDruidClient baseClient
|
||||
)
|
||||
{
|
||||
this.warehouse = warehouse;
|
||||
this.baseClient = baseClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
|
||||
{
|
||||
return makeRunner(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
|
||||
{
|
||||
return makeRunner(query);
|
||||
}
|
||||
|
||||
private <T> FinalizeResultsQueryRunner<T> makeRunner(final Query<T> query)
|
||||
{
|
||||
return new FinalizeResultsQueryRunner<T>(baseClient, warehouse.getToolChest(query));
|
||||
}
|
||||
}
|
|
@ -148,6 +148,7 @@ public class QueryServlet extends HttpServlet
|
|||
ImmutableMap.<String, Object>builder()
|
||||
.put("exception", e.toString())
|
||||
.put("query", queryString)
|
||||
.put("host", req.getRemoteAddr())
|
||||
.build()
|
||||
)
|
||||
);
|
||||
|
|
|
@ -203,7 +203,7 @@ public class Initialization
|
|||
log.info("Loaded(properties stored in zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop));
|
||||
}
|
||||
} // get props from zk
|
||||
} else { // ToDo: should this be an error?
|
||||
} else {
|
||||
log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination.");
|
||||
}
|
||||
// validate properties now that all levels of precedence are loaded
|
||||
|
|
|
@ -22,16 +22,19 @@ package com.metamx.druid.query;
|
|||
import com.google.common.base.Function;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.druid.Query;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheStrategy<T, QueryType extends Query<T>>
|
||||
public interface CacheStrategy<T, CacheType, QueryType extends Query<T>>
|
||||
{
|
||||
public byte[] computeCacheKey(QueryType query);
|
||||
|
||||
public Function<T, Object> prepareForCache();
|
||||
public TypeReference<CacheType> getCacheObjectClazz();
|
||||
|
||||
public Function<Object, T> pullFromCache();
|
||||
public Function<T, CacheType> prepareForCache();
|
||||
|
||||
public Function<CacheType, T> pullFromCache();
|
||||
|
||||
public Sequence<T> mergeSequences(Sequence<Sequence<T>> seqOfSequences);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid.query;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
@ -58,9 +59,12 @@ public class Queries
|
|||
);
|
||||
|
||||
for (PostAggregator postAgg : postAggs) {
|
||||
Set<String> dependencies = postAgg.getDependentFields();
|
||||
Set<String> missing = Sets.difference(dependencies, combinedAggNames);
|
||||
|
||||
Preconditions.checkArgument(
|
||||
postAgg.verifyFields(combinedAggNames),
|
||||
String.format("Missing field[%s]", postAgg.getName())
|
||||
missing.isEmpty(),
|
||||
"Missing fields [%s] for postAggregator [%s]", missing, postAgg.getName()
|
||||
);
|
||||
combinedAggNames.add(postAgg.getName());
|
||||
}
|
||||
|
|
|
@ -44,7 +44,7 @@ public interface QueryToolChest<ResultType, QueryType extends Query<ResultType>>
|
|||
public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
|
||||
public Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
|
||||
public TypeReference<ResultType> getResultTypeReference();
|
||||
public CacheStrategy<ResultType, QueryType> getCacheStrategy(QueryType query);
|
||||
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query);
|
||||
public QueryRunner<ResultType> preMergeQueryDecoration(QueryRunner<ResultType> runner);
|
||||
public QueryRunner<ResultType> postMergeQueryDecoration(QueryRunner<ResultType> runner);
|
||||
}
|
||||
|
|
|
@ -29,9 +29,11 @@ import com.metamx.common.guava.ConcatSequence;
|
|||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.index.v1.IncrementalIndex;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.input.MapBasedRow;
|
||||
import com.metamx.druid.input.Row;
|
||||
import com.metamx.druid.input.Rows;
|
||||
import com.metamx.druid.query.CacheStrategy;
|
||||
|
@ -99,10 +101,11 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
|
|||
}
|
||||
);
|
||||
|
||||
final QueryGranularity gran = query.getGranularity();
|
||||
final IncrementalIndex index = runner.run(query).accumulate(
|
||||
new IncrementalIndex(
|
||||
condensed.get(0).getStartMillis(),
|
||||
query.getGranularity(),
|
||||
gran.truncate(condensed.get(0).getStartMillis()),
|
||||
gran,
|
||||
aggs.toArray(new AggregatorFactory[aggs.size()])
|
||||
),
|
||||
new Accumulator<IncrementalIndex, Row>()
|
||||
|
@ -119,7 +122,21 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
|
|||
}
|
||||
);
|
||||
|
||||
return Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs()));
|
||||
// convert millis back to timestamp according to granularity to preserve time zone information
|
||||
return Sequences.map(
|
||||
Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())),
|
||||
new Function<Row, Row>()
|
||||
{
|
||||
private final QueryGranularity granularity = query.getGranularity();
|
||||
|
||||
@Override
|
||||
public Row apply(Row input)
|
||||
{
|
||||
final MapBasedRow row = (MapBasedRow) input;
|
||||
return new MapBasedRow(granularity.toDateTime(row.getTimestampFromEpoch()), row.getEvent());
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -161,7 +178,7 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
|
|||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Row, GroupByQuery> getCacheStrategy(GroupByQuery query)
|
||||
public CacheStrategy<Row, Object, GroupByQuery> getCacheStrategy(GroupByQuery query)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class AllColumnIncluderator implements ColumnIncluderator
|
||||
{
|
||||
@Override
|
||||
public boolean include(String columnName)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return ALL_CACHE_PREFIX;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* 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.Preconditions;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnAnalysis
|
||||
{
|
||||
private static final String ERROR_PREFIX = "error:";
|
||||
|
||||
public static ColumnAnalysis error(String reason)
|
||||
{
|
||||
return new ColumnAnalysis(ERROR_PREFIX + reason, -1, null);
|
||||
}
|
||||
|
||||
private final String type;
|
||||
private final long size;
|
||||
private final Integer cardinality;
|
||||
|
||||
@JsonCreator
|
||||
public ColumnAnalysis(
|
||||
@JsonProperty("type") ValueType type,
|
||||
@JsonProperty("size") long size,
|
||||
@JsonProperty("cardinality") Integer cardinality
|
||||
)
|
||||
{
|
||||
this(type.name(), size, cardinality);
|
||||
}
|
||||
|
||||
private ColumnAnalysis(
|
||||
String type,
|
||||
long size,
|
||||
Integer cardinality
|
||||
)
|
||||
{
|
||||
this.type = type;
|
||||
this.size = size;
|
||||
this.cardinality = cardinality;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getSize()
|
||||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Integer getCardinality()
|
||||
{
|
||||
return cardinality;
|
||||
}
|
||||
|
||||
public boolean isError()
|
||||
{
|
||||
return type.startsWith(ERROR_PREFIX);
|
||||
}
|
||||
|
||||
public ColumnAnalysis fold(ColumnAnalysis rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return this;
|
||||
}
|
||||
|
||||
if (!type.equals(rhs.getType())) {
|
||||
return ColumnAnalysis.error("cannot_merge_diff_types");
|
||||
}
|
||||
|
||||
Integer cardinality = getCardinality();
|
||||
final Integer rhsCardinality = rhs.getCardinality();
|
||||
if (cardinality == null) {
|
||||
cardinality = rhsCardinality;
|
||||
}
|
||||
else {
|
||||
if (rhsCardinality != null) {
|
||||
cardinality = Math.max(cardinality, rhsCardinality);
|
||||
}
|
||||
}
|
||||
|
||||
return new ColumnAnalysis(type, size + rhs.getSize(), cardinality);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ColumnAnalysis{" +
|
||||
"type='" + type + '\'' +
|
||||
", size=" + size +
|
||||
", cardinality=" + cardinality +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* 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 org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "none", value= NoneColumnIncluderator.class),
|
||||
@JsonSubTypes.Type(name = "all", value= AllColumnIncluderator.class),
|
||||
@JsonSubTypes.Type(name = "list", value= ListColumnIncluderator.class)
|
||||
})
|
||||
public interface ColumnIncluderator
|
||||
{
|
||||
public static final byte[] NONE_CACHE_PREFIX = new byte[]{0x0};
|
||||
public static final byte[] ALL_CACHE_PREFIX = new byte[]{0x1};
|
||||
public static final byte[] LIST_CACHE_PREFIX = new byte[]{0x2};
|
||||
|
||||
public boolean include(String columnName);
|
||||
public byte[] getCacheKey();
|
||||
}
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* 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.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ListColumnIncluderator implements ColumnIncluderator
|
||||
{
|
||||
private final Set<String> columns;
|
||||
|
||||
@JsonCreator
|
||||
public ListColumnIncluderator(
|
||||
@JsonProperty("columns") List<String> columns
|
||||
)
|
||||
{
|
||||
this.columns = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
|
||||
this.columns.addAll(columns);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<String> getColumns()
|
||||
{
|
||||
return Collections.unmodifiableSet(columns);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean include(String columnName)
|
||||
{
|
||||
return columns.contains(columnName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
int size = 1;
|
||||
List<byte[]> columns = Lists.newArrayListWithExpectedSize(this.columns.size());
|
||||
|
||||
for (String column : this.columns) {
|
||||
final byte[] bytes = column.getBytes(Charsets.UTF_8);
|
||||
columns.add(bytes);
|
||||
size += bytes.length;
|
||||
}
|
||||
|
||||
final ByteBuffer bytes = ByteBuffer.allocate(size).put(LIST_CACHE_PREFIX);
|
||||
for (byte[] column : columns) {
|
||||
bytes.put(column);
|
||||
}
|
||||
|
||||
return bytes.array();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NoneColumnIncluderator implements ColumnIncluderator
|
||||
{
|
||||
@Override
|
||||
public boolean include(String columnName)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return NONE_CACHE_PREFIX;
|
||||
}
|
||||
}
|
|
@ -17,61 +17,34 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.result;
|
||||
package com.metamx.druid.query.metadata;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class SegmentMetadataResultValue
|
||||
public class SegmentAnalysis
|
||||
{
|
||||
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<String, Dimension> dimensions;
|
||||
private final Map<String, Metric> metrics;
|
||||
private final List<Interval> interval;
|
||||
private final Map<String, ColumnAnalysis> columns;
|
||||
private final long size;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentMetadataResultValue(
|
||||
public SegmentAnalysis(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("dimensions") Map<String, Dimension> dimensions,
|
||||
@JsonProperty("metrics") Map<String, Metric> metrics,
|
||||
@JsonProperty("intervals") List<Interval> interval,
|
||||
@JsonProperty("columns") Map<String, ColumnAnalysis> columns,
|
||||
@JsonProperty("size") long size
|
||||
|
||||
)
|
||||
{
|
||||
this.id = id;
|
||||
this.dimensions = dimensions;
|
||||
this.metrics = metrics;
|
||||
this.interval = interval;
|
||||
this.columns = columns;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
|
@ -82,15 +55,15 @@ public class SegmentMetadataResultValue
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Dimension> getDimensions()
|
||||
public List<Interval> getIntervals()
|
||||
{
|
||||
return dimensions;
|
||||
return interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Metric> getMetrics()
|
||||
public Map<String, ColumnAnalysis> getColumns()
|
||||
{
|
||||
return metrics;
|
||||
return columns;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -98,4 +71,24 @@ public class SegmentMetadataResultValue
|
|||
{
|
||||
return size;
|
||||
}
|
||||
|
||||
public String toDetailedString()
|
||||
{
|
||||
return "SegmentAnalysis{" +
|
||||
"id='" + id + '\'' +
|
||||
", interval=" + interval +
|
||||
", columns=" + columns +
|
||||
", size=" + size +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "SegmentAnalysis{" +
|
||||
"id='" + id + '\'' +
|
||||
", interval=" + interval +
|
||||
", size=" + size +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -22,26 +22,40 @@ 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<Result<SegmentMetadataResultValue>>
|
||||
public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||
{
|
||||
|
||||
private final ColumnIncluderator toInclude;
|
||||
private final boolean merge;
|
||||
|
||||
public SegmentMetadataQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||
@JsonProperty("toInclude") ColumnIncluderator toInclude,
|
||||
@JsonProperty("merge") Boolean merge,
|
||||
@JsonProperty("context") Map<String, String> context
|
||||
)
|
||||
{
|
||||
super(
|
||||
dataSource,
|
||||
querySegmentSpec,
|
||||
context
|
||||
);
|
||||
super(dataSource, querySegmentSpec, context);
|
||||
|
||||
this.toInclude = toInclude == null ? new AllColumnIncluderator() : toInclude;
|
||||
this.merge = merge == null ? false : merge;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ColumnIncluderator getToInclude()
|
||||
{
|
||||
return toInclude;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isMerge()
|
||||
{
|
||||
return merge;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -57,22 +71,16 @@ public class SegmentMetadataQuery extends BaseQuery<Result<SegmentMetadataResult
|
|||
}
|
||||
|
||||
@Override
|
||||
public Query<Result<SegmentMetadataResultValue>> withOverriddenContext(Map<String, String> contextOverride)
|
||||
public Query<SegmentAnalysis> withOverriddenContext(Map<String, String> contextOverride)
|
||||
{
|
||||
return new SegmentMetadataQuery(
|
||||
getDataSource(),
|
||||
getQuerySegmentSpec(),
|
||||
computeOverridenContext(contextOverride)
|
||||
getDataSource(), getQuerySegmentSpec(), toInclude, merge, computeOverridenContext(contextOverride)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query<Result<SegmentMetadataResultValue>> withQuerySegmentSpec(QuerySegmentSpec spec)
|
||||
public Query<SegmentAnalysis> withQuerySegmentSpec(QuerySegmentSpec spec)
|
||||
{
|
||||
return new SegmentMetadataQuery(
|
||||
getDataSource(),
|
||||
spec,
|
||||
getContext()
|
||||
);
|
||||
return new SegmentMetadataQuery(getDataSource(), spec, toInclude, merge, getContext());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,32 +22,116 @@ 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.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.MergeSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.collect.OrderedMergeSequence;
|
||||
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.druid.query.ResultMergeQueryRunner;
|
||||
import com.metamx.druid.utils.JodaUtils;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<Result<SegmentMetadataResultValue>, SegmentMetadataQuery>
|
||||
|
||||
public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
||||
{
|
||||
|
||||
private static final TypeReference<Result<SegmentMetadataResultValue>> TYPE_REFERENCE = new TypeReference<Result<SegmentMetadataResultValue>>(){};
|
||||
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>(){};
|
||||
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<SegmentMetadataResultValue>> mergeResults(final QueryRunner<Result<SegmentMetadataResultValue>> runner)
|
||||
public QueryRunner<SegmentAnalysis> mergeResults(final QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
return new ConcatQueryRunner<Result<SegmentMetadataResultValue>>(Sequences.simple(ImmutableList.of(runner)));
|
||||
return new ResultMergeQueryRunner<SegmentAnalysis>(runner)
|
||||
{
|
||||
@Override
|
||||
protected Ordering<SegmentAnalysis> makeOrdering(Query<SegmentAnalysis> query)
|
||||
{
|
||||
if (((SegmentMetadataQuery) query).isMerge()) {
|
||||
// Merge everything always
|
||||
return new Ordering<SegmentAnalysis>()
|
||||
{
|
||||
@Override
|
||||
public int compare(
|
||||
@Nullable SegmentAnalysis left, @Nullable SegmentAnalysis right
|
||||
)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return getOrdering(); // No two elements should be equal, so it should never merge
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BinaryFn<SegmentAnalysis, SegmentAnalysis, SegmentAnalysis> createMergeFn(final Query<SegmentAnalysis> inQ)
|
||||
{
|
||||
return new BinaryFn<SegmentAnalysis, SegmentAnalysis, SegmentAnalysis>()
|
||||
{
|
||||
private final SegmentMetadataQuery query = (SegmentMetadataQuery) inQ;
|
||||
|
||||
@Override
|
||||
public SegmentAnalysis apply(SegmentAnalysis arg1, SegmentAnalysis arg2)
|
||||
{
|
||||
if (arg1 == null) {
|
||||
return arg2;
|
||||
}
|
||||
|
||||
if (arg2 == null) {
|
||||
return arg1;
|
||||
}
|
||||
|
||||
if (!query.isMerge()) {
|
||||
throw new ISE("Merging when a merge isn't supposed to happen[%s], [%s]", arg1, arg2);
|
||||
}
|
||||
|
||||
List<Interval> newIntervals = JodaUtils.condenseIntervals(
|
||||
Iterables.concat(arg1.getIntervals(), arg2.getIntervals())
|
||||
);
|
||||
|
||||
final Map<String, ColumnAnalysis> leftColumns = arg1.getColumns();
|
||||
final Map<String, ColumnAnalysis> rightColumns = arg2.getColumns();
|
||||
Map<String, ColumnAnalysis> columns = Maps.newTreeMap();
|
||||
|
||||
Set<String> rightColumnNames = Sets.newHashSet(rightColumns.keySet());
|
||||
for (Map.Entry<String, ColumnAnalysis> entry : leftColumns.entrySet()) {
|
||||
final String columnName = entry.getKey();
|
||||
columns.put(columnName, entry.getValue().fold(rightColumns.get(columnName)));
|
||||
rightColumnNames.remove(columnName);
|
||||
}
|
||||
|
||||
for (String columnName : rightColumnNames) {
|
||||
columns.put(columnName, rightColumns.get(columnName));
|
||||
}
|
||||
|
||||
return new SegmentAnalysis("merged", newIntervals, columns, arg1.getSize() + arg2.getSize());
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<SegmentAnalysis> mergeSequences(Sequence<Sequence<SegmentAnalysis>> seqOfSequences)
|
||||
{
|
||||
return new OrderedMergeSequence<SegmentAnalysis>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -67,13 +151,7 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<Result
|
|||
}
|
||||
|
||||
@Override
|
||||
public Sequence<Result<SegmentMetadataResultValue>> mergeSequences(Sequence<Sequence<Result<SegmentMetadataResultValue>>> seqOfSequences)
|
||||
{
|
||||
return new ConcatSequence<Result<SegmentMetadataResultValue>>(seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<SegmentMetadataResultValue>, Result<SegmentMetadataResultValue>> makeMetricManipulatorFn(
|
||||
public Function<SegmentAnalysis, SegmentAnalysis> makeMetricManipulatorFn(
|
||||
SegmentMetadataQuery query, MetricManipulationFn fn
|
||||
)
|
||||
{
|
||||
|
@ -81,26 +159,87 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<Result
|
|||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Result<SegmentMetadataResultValue>> getResultTypeReference()
|
||||
public TypeReference<SegmentAnalysis> getResultTypeReference()
|
||||
{
|
||||
return TYPE_REFERENCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Result<SegmentMetadataResultValue>, SegmentMetadataQuery> getCacheStrategy(SegmentMetadataQuery query)
|
||||
public CacheStrategy<SegmentAnalysis, SegmentAnalysis, SegmentMetadataQuery> getCacheStrategy(SegmentMetadataQuery query)
|
||||
{
|
||||
return null;
|
||||
return new CacheStrategy<SegmentAnalysis, SegmentAnalysis, SegmentMetadataQuery>()
|
||||
{
|
||||
@Override
|
||||
public byte[] computeCacheKey(SegmentMetadataQuery query)
|
||||
{
|
||||
byte[] includerBytes = query.getToInclude().getCacheKey();
|
||||
return ByteBuffer.allocate(1 + includerBytes.length)
|
||||
.put(SEGMENT_METADATA_CACHE_PREFIX)
|
||||
.put(includerBytes)
|
||||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<SegmentAnalysis> getCacheObjectClazz()
|
||||
{
|
||||
return getResultTypeReference();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentAnalysis, SegmentAnalysis> prepareForCache()
|
||||
{
|
||||
return new Function<SegmentAnalysis, SegmentAnalysis>()
|
||||
{
|
||||
@Override
|
||||
public SegmentAnalysis apply(@Nullable SegmentAnalysis input)
|
||||
{
|
||||
return input;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentAnalysis, SegmentAnalysis> pullFromCache()
|
||||
{
|
||||
return new Function<SegmentAnalysis, SegmentAnalysis>()
|
||||
{
|
||||
@Override
|
||||
public SegmentAnalysis apply(@Nullable SegmentAnalysis input)
|
||||
{
|
||||
return input;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<SegmentAnalysis> mergeSequences(Sequence<Sequence<SegmentAnalysis>> seqOfSequences)
|
||||
{
|
||||
return new MergeSequence<SegmentAnalysis>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<SegmentMetadataResultValue>> preMergeQueryDecoration(QueryRunner<Result<SegmentMetadataResultValue>> runner)
|
||||
public QueryRunner<SegmentAnalysis> preMergeQueryDecoration(QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
return runner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<SegmentMetadataResultValue>> postMergeQueryDecoration(QueryRunner<Result<SegmentMetadataResultValue>> runner)
|
||||
public QueryRunner<SegmentAnalysis> postMergeQueryDecoration(QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
return runner;
|
||||
}
|
||||
|
||||
private Ordering<SegmentAnalysis> getOrdering()
|
||||
{
|
||||
return new Ordering<SegmentAnalysis>()
|
||||
{
|
||||
@Override
|
||||
public int compare(SegmentAnalysis left, SegmentAnalysis right)
|
||||
{
|
||||
return left.getId().compareTo(right.getId());
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -82,6 +82,10 @@ public class SearchQueryQueryToolChest implements QueryToolChest<Result<SearchRe
|
|||
maxSearchLimit = PropUtils.getPropertyAsInt(props, "com.metamx.query.search.maxSearchLimit", 1000);
|
||||
}
|
||||
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE = new TypeReference<Object>()
|
||||
{
|
||||
};
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<SearchResultValue>> mergeResults(QueryRunner<Result<SearchResultValue>> runner)
|
||||
{
|
||||
|
@ -143,9 +147,9 @@ public class SearchQueryQueryToolChest implements QueryToolChest<Result<SearchRe
|
|||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Result<SearchResultValue>, SearchQuery> getCacheStrategy(SearchQuery query)
|
||||
public CacheStrategy<Result<SearchResultValue>, Object, SearchQuery> getCacheStrategy(SearchQuery query)
|
||||
{
|
||||
return new CacheStrategy<Result<SearchResultValue>, SearchQuery>()
|
||||
return new CacheStrategy<Result<SearchResultValue>, Object, SearchQuery>()
|
||||
{
|
||||
@Override
|
||||
public byte[] computeCacheKey(SearchQuery query)
|
||||
|
@ -183,6 +187,12 @@ public class SearchQueryQueryToolChest implements QueryToolChest<Result<SearchRe
|
|||
return queryCacheKey.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Object> getCacheObjectClazz()
|
||||
{
|
||||
return OBJECT_TYPE_REFERENCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<SearchResultValue>, Object> prepareForCache()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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 org.joda.time.Interval;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QuerySegmentSpecs
|
||||
{
|
||||
public static QuerySegmentSpec create(String isoInterval)
|
||||
{
|
||||
return new LegacySegmentSpec(isoInterval);
|
||||
}
|
||||
|
||||
public static QuerySegmentSpec create(Interval interval)
|
||||
{
|
||||
return create(Arrays.asList(interval));
|
||||
}
|
||||
|
||||
public static QuerySegmentSpec create(List<Interval> intervals)
|
||||
{
|
||||
return new MultipleIntervalSegmentSpec(intervals);
|
||||
}
|
||||
}
|
|
@ -53,6 +53,9 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
private static final TypeReference<Result<TimeBoundaryResultValue>> TYPE_REFERENCE = new TypeReference<Result<TimeBoundaryResultValue>>()
|
||||
{
|
||||
};
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE = new TypeReference<Object>()
|
||||
{
|
||||
};
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<TimeBoundaryResultValue>> mergeResults(
|
||||
|
@ -106,9 +109,9 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Result<TimeBoundaryResultValue>, TimeBoundaryQuery> getCacheStrategy(TimeBoundaryQuery query)
|
||||
public CacheStrategy<Result<TimeBoundaryResultValue>, Object, TimeBoundaryQuery> getCacheStrategy(TimeBoundaryQuery query)
|
||||
{
|
||||
return new CacheStrategy<Result<TimeBoundaryResultValue>, TimeBoundaryQuery>()
|
||||
return new CacheStrategy<Result<TimeBoundaryResultValue>, Object, TimeBoundaryQuery>()
|
||||
{
|
||||
@Override
|
||||
public byte[] computeCacheKey(TimeBoundaryQuery query)
|
||||
|
@ -119,6 +122,12 @@ public class TimeBoundaryQueryQueryToolChest
|
|||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Object> getCacheObjectClazz()
|
||||
{
|
||||
return OBJECT_TYPE_REFERENCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeBoundaryResultValue>, Object> prepareForCache()
|
||||
{
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.guava.MergeSequence;
|
|||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.nary.BinaryFn;
|
||||
import com.metamx.druid.Query;
|
||||
import com.metamx.druid.QueryGranularity;
|
||||
import com.metamx.druid.ResultGranularTimestampComparator;
|
||||
import com.metamx.druid.TimeseriesBinaryFn;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
|
@ -49,6 +50,7 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
import org.joda.time.Minutes;
|
||||
import org.joda.time.Period;
|
||||
import org.joda.time.format.ISODateTimeFormat;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -66,6 +68,9 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
private static final TypeReference<Result<TimeseriesResultValue>> TYPE_REFERENCE = new TypeReference<Result<TimeseriesResultValue>>()
|
||||
{
|
||||
};
|
||||
private static final TypeReference<Object> OBJECT_TYPE_REFERENCE = new TypeReference<Object>()
|
||||
{
|
||||
};
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<TimeseriesResultValue>> mergeResults(QueryRunner<Result<TimeseriesResultValue>> queryRunner)
|
||||
|
@ -100,10 +105,7 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
@Override
|
||||
public Sequence<Result<TimeseriesResultValue>> mergeSequences(Sequence<Sequence<Result<TimeseriesResultValue>>> seqOfSequences)
|
||||
{
|
||||
return new OrderedMergeSequence<Result<TimeseriesResultValue>>(
|
||||
getOrdering(),
|
||||
seqOfSequences
|
||||
);
|
||||
return new OrderedMergeSequence<Result<TimeseriesResultValue>>(getOrdering(), seqOfSequences);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -156,9 +158,9 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
}
|
||||
|
||||
@Override
|
||||
public CacheStrategy<Result<TimeseriesResultValue>, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query)
|
||||
public CacheStrategy<Result<TimeseriesResultValue>, Object, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query)
|
||||
{
|
||||
return new CacheStrategy<Result<TimeseriesResultValue>, TimeseriesQuery>()
|
||||
return new CacheStrategy<Result<TimeseriesResultValue>, Object, TimeseriesQuery>()
|
||||
{
|
||||
private final List<AggregatorFactory> aggs = query.getAggregatorSpecs();
|
||||
private final List<PostAggregator> postAggs = query.getPostAggregatorSpecs();
|
||||
|
@ -180,6 +182,12 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TypeReference<Object> getCacheObjectClazz()
|
||||
{
|
||||
return OBJECT_TYPE_REFERENCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<Result<TimeseriesResultValue>, Object> prepareForCache()
|
||||
{
|
||||
|
@ -206,6 +214,8 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
{
|
||||
return new Function<Object, Result<TimeseriesResultValue>>()
|
||||
{
|
||||
private final QueryGranularity granularity = query.getGranularity();
|
||||
|
||||
@Override
|
||||
public Result<TimeseriesResultValue> apply(@Nullable Object input)
|
||||
{
|
||||
|
@ -215,7 +225,8 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
Iterator<AggregatorFactory> aggsIter = aggs.iterator();
|
||||
Iterator<Object> resultIter = results.iterator();
|
||||
|
||||
DateTime timestamp = new DateTime(resultIter.next());
|
||||
DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue());
|
||||
|
||||
while (aggsIter.hasNext() && resultIter.hasNext()) {
|
||||
final AggregatorFactory factory = aggsIter.next();
|
||||
retVal.put(factory.getName(), factory.deserialize(resultIter.next()));
|
||||
|
@ -257,6 +268,4 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
|
|||
{
|
||||
return Ordering.natural();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -71,7 +71,7 @@ public class Result<T> implements Comparable<Result<T>>
|
|||
|
||||
Result result = (Result) o;
|
||||
|
||||
if (timestamp != null ? !timestamp.equals(result.timestamp) : result.timestamp != null) {
|
||||
if (timestamp != null ? !(timestamp.isEqual(result.timestamp) && timestamp.getZone().getOffset(timestamp) == result.timestamp.getZone().getOffset(result.timestamp)) : result.timestamp != null) {
|
||||
return false;
|
||||
}
|
||||
if (value != null ? !value.equals(result.value) : result.value != null) {
|
||||
|
|
|
@ -112,7 +112,8 @@ class StoppedPhoneBook implements PhoneBook
|
|||
}
|
||||
|
||||
if (! serviceAnnouncements.containsKey(nodeName)) {
|
||||
throw new IAE("Cannot unannounce node[%s] on service[%s]", nodeName, serviceName);
|
||||
log.warn("Cannot unannounce[%s]: it doesn't exist for service[%s]", nodeName, serviceName);
|
||||
return;
|
||||
}
|
||||
|
||||
serviceAnnouncements.remove(nodeName);
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.druid.index.v1.IndexIO;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.druid.shard.SingleDimensionShardSpec;
|
||||
|
@ -60,12 +61,13 @@ public class DataSegmentTest
|
|||
Arrays.asList("dim1", "dim2"),
|
||||
Arrays.asList("met1", "met2"),
|
||||
new NoneShardSpec(),
|
||||
IndexIO.CURRENT_VERSION_ID,
|
||||
1
|
||||
);
|
||||
|
||||
final Map<String, Object> objectMap = mapper.readValue(mapper.writeValueAsString(segment), new TypeReference<Map<String, Object>>(){});
|
||||
|
||||
Assert.assertEquals(9, objectMap.size());
|
||||
Assert.assertEquals(10, objectMap.size());
|
||||
Assert.assertEquals("something", objectMap.get("dataSource"));
|
||||
Assert.assertEquals(interval.toString(), objectMap.get("interval"));
|
||||
Assert.assertEquals("1", objectMap.get("version"));
|
||||
|
@ -73,6 +75,7 @@ public class DataSegmentTest
|
|||
Assert.assertEquals("dim1,dim2", objectMap.get("dimensions"));
|
||||
Assert.assertEquals("met1,met2", objectMap.get("metrics"));
|
||||
Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec"));
|
||||
Assert.assertEquals(IndexIO.CURRENT_VERSION_ID, objectMap.get("binaryVersion"));
|
||||
Assert.assertEquals(1, objectMap.get("size"));
|
||||
|
||||
DataSegment deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
|
||||
|
|
|
@ -26,61 +26,58 @@ import org.junit.Test;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class MapCacheBrokerTest
|
||||
public class MapCacheTest
|
||||
{
|
||||
private static final byte[] HI = "hi".getBytes();
|
||||
private static final byte[] HO = "ho".getBytes();
|
||||
private ByteCountingLRUMap baseMap;
|
||||
private MapCacheBroker broker;
|
||||
private MapCache cache;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
baseMap = new ByteCountingLRUMap(1024 * 1024);
|
||||
broker = new MapCacheBroker(baseMap);
|
||||
cache = new MapCache(baseMap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSanity() throws Exception
|
||||
{
|
||||
Cache aCache = broker.provideCache("a");
|
||||
Cache theCache = broker.provideCache("the");
|
||||
|
||||
Assert.assertNull(aCache.get(HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HI)));
|
||||
Assert.assertEquals(0, baseMap.size());
|
||||
put(aCache, HI, 1);
|
||||
put(cache, "a", HI, 1);
|
||||
Assert.assertEquals(1, baseMap.size());
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(theCache.get(HI));
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("the", HI)));
|
||||
|
||||
put(theCache, HI, 2);
|
||||
put(cache, "the", HI, 2);
|
||||
Assert.assertEquals(2, baseMap.size());
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertEquals(2, get(theCache, HI));
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertEquals(2, get(cache, "the", HI));
|
||||
|
||||
put(theCache, HO, 10);
|
||||
put(cache, "the", HO, 10);
|
||||
Assert.assertEquals(3, baseMap.size());
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(aCache.get(HO));
|
||||
Assert.assertEquals(2, get(theCache, HI));
|
||||
Assert.assertEquals(10, get(theCache, HO));
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HO)));
|
||||
Assert.assertEquals(2, get(cache, "the", HI));
|
||||
Assert.assertEquals(10, get(cache, "the", HO));
|
||||
|
||||
theCache.close();
|
||||
cache.close("the");
|
||||
Assert.assertEquals(1, baseMap.size());
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(aCache.get(HO));
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HO)));
|
||||
|
||||
aCache.close();
|
||||
cache.close("a");
|
||||
Assert.assertEquals(0, baseMap.size());
|
||||
}
|
||||
|
||||
public void put(Cache cache, byte[] key, Integer value)
|
||||
public void put(Cache cache, String namespace, byte[] key, Integer value)
|
||||
{
|
||||
cache.put(key, Ints.toByteArray(value));
|
||||
cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value));
|
||||
}
|
||||
|
||||
public int get(Cache cache, byte[] key)
|
||||
public int get(Cache cache, String namespace, byte[] key)
|
||||
{
|
||||
return Ints.fromByteArray(cache.get(key));
|
||||
return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key)));
|
||||
}
|
||||
}
|
|
@ -3,6 +3,7 @@ package com.metamx.druid.client.cache;
|
|||
import com.google.caliper.Param;
|
||||
import com.google.caliper.Runner;
|
||||
import com.google.caliper.SimpleBenchmark;
|
||||
import com.google.common.collect.Lists;
|
||||
import net.spy.memcached.AddrUtil;
|
||||
import net.spy.memcached.ConnectionFactoryBuilder;
|
||||
import net.spy.memcached.DefaultHashAlgorithm;
|
||||
|
@ -11,17 +12,19 @@ import net.spy.memcached.MemcachedClient;
|
|||
import net.spy.memcached.MemcachedClientIF;
|
||||
import net.spy.memcached.transcoders.SerializingTranscoder;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark
|
||||
public class MemcachedCacheBenchmark extends SimpleBenchmark
|
||||
{
|
||||
private static final String BASE_KEY = "test_2012-11-26T00:00:00.000Z_2012-11-27T00:00:00.000Z_2012-11-27T04:11:25.979Z_";
|
||||
public static final String NAMESPACE = "default";
|
||||
|
||||
private MemcachedCacheBroker broker;
|
||||
private MemcachedCache cache;
|
||||
private MemcachedClientIF client;
|
||||
|
||||
private Cache cache;
|
||||
private static byte[] randBytes;
|
||||
|
||||
@Param({"localhost:11211"}) String hosts;
|
||||
|
@ -39,8 +42,6 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark
|
|||
// disable compression
|
||||
transcoder.setCompressionThreshold(Integer.MAX_VALUE);
|
||||
|
||||
System.out.println(String.format("Using memcached hosts [%s]", hosts));
|
||||
|
||||
client = new MemcachedClient(
|
||||
new ConnectionFactoryBuilder().setProtocol(ConnectionFactoryBuilder.Protocol.BINARY)
|
||||
.setHashAlg(DefaultHashAlgorithm.FNV1A_64_HASH)
|
||||
|
@ -53,14 +54,13 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark
|
|||
AddrUtil.getAddresses(hosts)
|
||||
);
|
||||
|
||||
broker = new MemcachedCacheBroker(
|
||||
cache = new MemcachedCache(
|
||||
client,
|
||||
500, // 500 milliseconds
|
||||
3600 * 24 * 365 // 1 year
|
||||
"druid-memcached-benchmark",
|
||||
30000, // 30 seconds
|
||||
3600 // 1 hour
|
||||
);
|
||||
|
||||
cache = broker.provideCache("default");
|
||||
|
||||
|
||||
randBytes = new byte[objectSize * 1024];
|
||||
new Random(0).nextBytes(randBytes);
|
||||
|
@ -69,33 +69,51 @@ public class MemcachedCacheBrokerBenchmark extends SimpleBenchmark
|
|||
@Override
|
||||
protected void tearDown() throws Exception
|
||||
{
|
||||
client.flush();
|
||||
client.shutdown();
|
||||
client.shutdown(1, TimeUnit.MINUTES);
|
||||
}
|
||||
|
||||
public void timePutObjects(int reps) {
|
||||
for(int i = 0; i < reps; ++i) {
|
||||
for(int k = 0; k < objectCount; ++k) {
|
||||
String key = BASE_KEY + i;
|
||||
cache.put(key.getBytes(), randBytes);
|
||||
String key = BASE_KEY + k;
|
||||
cache.put(new Cache.NamedKey(NAMESPACE, key.getBytes()), randBytes);
|
||||
}
|
||||
// make sure the write queue is empty
|
||||
client.waitForQueues(1, TimeUnit.HOURS);
|
||||
}
|
||||
}
|
||||
|
||||
public byte[] timeGetObject(int reps) {
|
||||
public long timeGetObject(int reps) {
|
||||
byte[] bytes = null;
|
||||
long count = 0;
|
||||
for (int i = 0; i < reps; i++) {
|
||||
for(int k = 0; k < objectCount; ++k) {
|
||||
String key = BASE_KEY + i;
|
||||
bytes = cache.get(key.getBytes());
|
||||
String key = BASE_KEY + k;
|
||||
bytes = cache.get(new Cache.NamedKey(NAMESPACE, key.getBytes()));
|
||||
count += bytes.length;
|
||||
}
|
||||
}
|
||||
return bytes;
|
||||
return count;
|
||||
}
|
||||
|
||||
public long timeBulkGetObjects(int reps) {
|
||||
long count = 0;
|
||||
for (int i = 0; i < reps; i++) {
|
||||
List<Cache.NamedKey> keys = Lists.newArrayList();
|
||||
for(int k = 0; k < objectCount; ++k) {
|
||||
String key = BASE_KEY + k;
|
||||
keys.add(new Cache.NamedKey(NAMESPACE, key.getBytes()));
|
||||
}
|
||||
Map<Cache.NamedKey, byte[]> results = cache.getBulk(keys);
|
||||
for(Cache.NamedKey key : keys) {
|
||||
byte[] bytes = results.get(key);
|
||||
count += bytes.length;
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
Runner.main(MemcachedCacheBrokerBenchmark.class, args);
|
||||
Runner.main(MemcachedCacheBenchmark.class, args);
|
||||
}
|
||||
}
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package com.metamx.druid.client.cache;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import net.spy.memcached.CASResponse;
|
||||
import net.spy.memcached.CASValue;
|
||||
|
@ -27,6 +29,7 @@ import net.spy.memcached.ConnectionObserver;
|
|||
import net.spy.memcached.MemcachedClientIF;
|
||||
import net.spy.memcached.NodeLocator;
|
||||
import net.spy.memcached.internal.BulkFuture;
|
||||
import net.spy.memcached.ops.OperationStatus;
|
||||
import net.spy.memcached.transcoders.SerializingTranscoder;
|
||||
import net.spy.memcached.transcoders.Transcoder;
|
||||
import org.junit.Assert;
|
||||
|
@ -47,55 +50,74 @@ import java.util.concurrent.TimeoutException;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class MemcachedCacheBrokerTest
|
||||
public class MemcachedCacheTest
|
||||
{
|
||||
private static final byte[] HI = "hi".getBytes();
|
||||
private static final byte[] HO = "ho".getBytes();
|
||||
private MemcachedCacheBroker broker;
|
||||
private MemcachedCache cache;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
MemcachedClientIF client = new MockMemcachedClient();
|
||||
broker = new MemcachedCacheBroker(client, 500, 3600);
|
||||
cache = new MemcachedCache(client, "druid-memcached-test", 500, 3600);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSanity() throws Exception
|
||||
{
|
||||
Cache aCache = broker.provideCache("a");
|
||||
Cache theCache = broker.provideCache("the");
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HI)));
|
||||
put(cache, "a", HI, 1);
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("the", HI)));
|
||||
|
||||
Assert.assertNull(aCache.get(HI));
|
||||
put(aCache, HI, 1);
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(theCache.get(HI));
|
||||
put(cache, "the", HI, 2);
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertEquals(2, get(cache, "the", HI));
|
||||
|
||||
put(theCache, HI, 2);
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertEquals(2, get(theCache, HI));
|
||||
put(cache, "the", HO, 10);
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HO)));
|
||||
Assert.assertEquals(2, get(cache, "the", HI));
|
||||
Assert.assertEquals(10, get(cache, "the", HO));
|
||||
|
||||
put(theCache, HO, 10);
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(aCache.get(HO));
|
||||
Assert.assertEquals(2, get(theCache, HI));
|
||||
Assert.assertEquals(10, get(theCache, HO));
|
||||
cache.close("the");
|
||||
Assert.assertEquals(1, get(cache, "a", HI));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("a", HO)));
|
||||
|
||||
theCache.close();
|
||||
Assert.assertEquals(1, get(aCache, HI));
|
||||
Assert.assertNull(aCache.get(HO));
|
||||
|
||||
aCache.close();
|
||||
cache.close("a");
|
||||
}
|
||||
|
||||
public void put(Cache cache, byte[] key, Integer value)
|
||||
@Test
|
||||
public void testGetBulk() throws Exception
|
||||
{
|
||||
cache.put(key, Ints.toByteArray(value));
|
||||
Assert.assertNull(cache.get(new Cache.NamedKey("the", HI)));
|
||||
|
||||
put(cache, "the", HI, 2);
|
||||
put(cache, "the", HO, 10);
|
||||
|
||||
Cache.NamedKey key1 = new Cache.NamedKey("the", HI);
|
||||
Cache.NamedKey key2 = new Cache.NamedKey("the", HO);
|
||||
|
||||
Map<Cache.NamedKey, byte[]> result = cache.getBulk(
|
||||
Lists.newArrayList(
|
||||
key1,
|
||||
key2
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(2, Ints.fromByteArray(result.get(key1)));
|
||||
Assert.assertEquals(10, Ints.fromByteArray(result.get(key2)));
|
||||
}
|
||||
|
||||
public int get(Cache cache, byte[] key)
|
||||
public void put(Cache cache, String namespace, byte[] key, Integer value)
|
||||
{
|
||||
return Ints.fromByteArray(cache.get(key));
|
||||
cache.put(new Cache.NamedKey(namespace, key), Ints.toByteArray(value));
|
||||
}
|
||||
|
||||
public int get(Cache cache, String namespace, byte[] key)
|
||||
{
|
||||
return Ints.fromByteArray(cache.get(new Cache.NamedKey(namespace, key)));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -365,9 +387,67 @@ class MockMemcachedClient implements MemcachedClientIF
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T> BulkFuture<Map<String, T>> asyncGetBulk(Iterator<String> keys, Transcoder<T> tc)
|
||||
public <T> BulkFuture<Map<String, T>> asyncGetBulk(final Iterator<String> keys, final Transcoder<T> tc)
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
return new BulkFuture<Map<String, T>>()
|
||||
{
|
||||
@Override
|
||||
public boolean isTimeout()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, T> getSome(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException
|
||||
{
|
||||
return get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperationStatus getStatus()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean cancel(boolean b)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCancelled()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDone()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, T> get() throws InterruptedException, ExecutionException
|
||||
{
|
||||
Map<String, T> retVal = Maps.newHashMap();
|
||||
|
||||
while(keys.hasNext()) {
|
||||
String key = keys.next();
|
||||
CachedData data = theMap.get(key);
|
||||
retVal.put(key, data != null ? tc.decode(data) : null);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, T> get(long l, TimeUnit timeUnit)
|
||||
throws InterruptedException, ExecutionException, TimeoutException
|
||||
{
|
||||
return get();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -383,9 +463,9 @@ class MockMemcachedClient implements MemcachedClientIF
|
|||
}
|
||||
|
||||
@Override
|
||||
public BulkFuture<Map<String, Object>> asyncGetBulk(Collection<String> keys)
|
||||
public BulkFuture<Map<String, Object>> asyncGetBulk(final Collection<String> keys)
|
||||
{
|
||||
throw new UnsupportedOperationException("not implemented");
|
||||
return asyncGetBulk(keys.iterator(), transcoder);
|
||||
}
|
||||
|
||||
@Override
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package com.metamx.druid.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.IAE;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
@ -69,14 +70,13 @@ public class ArithmeticPostAggregator implements PostAggregator
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean verifyFields(Set<String> fieldNames)
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
for (PostAggregator field : fields) {
|
||||
if (!field.verifyFields(fieldNames)) {
|
||||
return false;
|
||||
}
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
return true;
|
||||
return dependentFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package com.metamx.druid.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
|
@ -44,9 +45,9 @@ public class ConstantPostAggregator implements PostAggregator
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean verifyFields(Set<String> fields)
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
return true;
|
||||
return Sets.newHashSet();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package com.metamx.druid.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.ISE;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
@ -45,9 +46,9 @@ public class FieldAccessPostAggregator implements PostAggregator
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean verifyFields(Set<String> fieldNames)
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
return fieldNames.contains(fieldName);
|
||||
return Sets.newHashSet(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.Set;
|
|||
})
|
||||
public interface PostAggregator
|
||||
{
|
||||
public boolean verifyFields(Set<String> fieldNames);
|
||||
public Set<String> getDependentFields();
|
||||
|
||||
public Comparator getComparator();
|
||||
|
||||
|
|
|
@ -59,6 +59,18 @@ public class DbConnector
|
|||
);
|
||||
}
|
||||
|
||||
public static void createConfigTable(final DBI dbi, final String configTableName)
|
||||
{
|
||||
createTable(
|
||||
dbi,
|
||||
configTableName,
|
||||
String.format(
|
||||
"CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))",
|
||||
configTableName
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static void createTable(
|
||||
final DBI dbi,
|
||||
final String tableName,
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.Map;
|
|||
*/
|
||||
public class MapBasedRow implements Row
|
||||
{
|
||||
private final long timestamp;
|
||||
private final DateTime timestamp;
|
||||
private final Map<String, Object> event;
|
||||
|
||||
@JsonCreator
|
||||
|
@ -46,22 +46,21 @@ public class MapBasedRow implements Row
|
|||
@JsonProperty("event") Map<String, Object> event
|
||||
)
|
||||
{
|
||||
this(timestamp.getMillis(), event);
|
||||
this.timestamp = timestamp;
|
||||
this.event = event;
|
||||
}
|
||||
|
||||
public MapBasedRow(
|
||||
long timestamp,
|
||||
Map<String, Object> event
|
||||
)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.event = event;
|
||||
) {
|
||||
this(new DateTime(timestamp), event);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestampFromEpoch()
|
||||
{
|
||||
return timestamp;
|
||||
return timestamp.getMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -120,7 +119,7 @@ public class MapBasedRow implements Row
|
|||
@JsonProperty
|
||||
public DateTime getTimestamp()
|
||||
{
|
||||
return new DateTime(timestamp);
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -133,9 +132,38 @@ public class MapBasedRow implements Row
|
|||
public String toString()
|
||||
{
|
||||
return "MapBasedRow{" +
|
||||
"timestamp=" + new DateTime(timestamp) +
|
||||
"timestamp=" + timestamp +
|
||||
", event=" + event +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
MapBasedRow that = (MapBasedRow) o;
|
||||
|
||||
if (!event.equals(that.event)) {
|
||||
return false;
|
||||
}
|
||||
if (!timestamp.equals(that.timestamp)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = timestamp.hashCode();
|
||||
result = 31 * result + event.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,12 @@ public class Rows
|
|||
{
|
||||
return row.getFloatMetric(metric);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return row.toString();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,10 +35,13 @@ import org.codehaus.jackson.map.JsonSerializer;
|
|||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.SerializationConfig;
|
||||
import org.codehaus.jackson.map.SerializerProvider;
|
||||
import org.codehaus.jackson.map.Serializers;
|
||||
import org.codehaus.jackson.map.module.SimpleModule;
|
||||
import org.codehaus.jackson.map.ser.std.ToStringSerializer;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.TimeZone;
|
||||
|
||||
/**
|
||||
|
@ -131,6 +134,23 @@ public class DefaultObjectMapper extends ObjectMapper
|
|||
}
|
||||
}
|
||||
);
|
||||
serializerModule.addSerializer(ByteOrder.class, ToStringSerializer.instance);
|
||||
serializerModule.addDeserializer(
|
||||
ByteOrder.class,
|
||||
new JsonDeserializer<ByteOrder>()
|
||||
{
|
||||
@Override
|
||||
public ByteOrder deserialize(
|
||||
JsonParser jp, DeserializationContext ctxt
|
||||
) throws IOException, JsonProcessingException
|
||||
{
|
||||
if (ByteOrder.BIG_ENDIAN.toString().equals(jp.getText())) {
|
||||
return ByteOrder.BIG_ENDIAN;
|
||||
}
|
||||
return ByteOrder.LITTLE_ENDIAN;
|
||||
}
|
||||
}
|
||||
);
|
||||
registerModule(serializerModule);
|
||||
|
||||
configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.processing;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Factory class for MetricSelectors
|
||||
*/
|
||||
|
|
|
@ -24,11 +24,11 @@
|
|||
<artifactId>druid-services</artifactId>
|
||||
<name>druid-services</name>
|
||||
<description>druid-services</description>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<modules>
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -11,7 +11,7 @@ import com.metamx.druid.log.LogLevelAdjuster;
|
|||
import com.metamx.druid.realtime.MetadataUpdater;
|
||||
import com.metamx.druid.realtime.MetadataUpdaterConfig;
|
||||
import com.metamx.druid.realtime.RealtimeNode;
|
||||
import com.metamx.druid.realtime.SegmentPusher;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import org.codehaus.jackson.map.jsontype.NamedType;
|
||||
|
||||
|
@ -21,8 +21,6 @@ import java.io.IOException;
|
|||
/**
|
||||
* Standalone Demo Realtime process.
|
||||
* Created: 20121009T2050
|
||||
*
|
||||
* @author pbaclace
|
||||
*/
|
||||
public class RealtimeStandaloneMain
|
||||
{
|
||||
|
|
|
@ -41,14 +41,6 @@ druid.paths.segmentInfoCache=/tmp/rand_realtime/segmentInfoCache
|
|||
# Path to schema definition file
|
||||
druid.request.logging.dir=/tmp/rand_realtime/log
|
||||
|
||||
# TODO: have these moved to spec file?
|
||||
# unknown # druid.realtime.dataSources=
|
||||
# unknown # druid.realtime.index.maxSize=500000
|
||||
# unknown # druid.realtime.persistPeriod=PT600S
|
||||
# unknown # druid.realtime.scheduledExec.threads=1
|
||||
# unknown # druid.realtime.uploadPeriod=PT3600S
|
||||
# unknown # druid.realtime.windowPeriod=PT600S
|
||||
|
||||
#druid.server.maxSize=0
|
||||
druid.server.maxSize=300000000000
|
||||
# =realtime or =historical (default)
|
||||
|
|
|
@ -2,19 +2,11 @@
|
|||
"queryType": "groupBy",
|
||||
"dataSource": "twitterstream",
|
||||
"granularity": "all",
|
||||
"dimensions": ["lang"],
|
||||
"dimensions": ["lang", "utc_offset"],
|
||||
"aggregations":[
|
||||
{ "type": "count", "name": "rows"},
|
||||
{ "type": "doubleSum", "fieldName": "tweets", "name": "tweets"},
|
||||
|
||||
{ "type": "max", "fieldName": "max_statuses_count", "name": "theMaxStatusesCount"},
|
||||
{ "type": "max", "fieldName": "max_retweet_count", "name": "theMaxRetweetCount"},
|
||||
|
||||
{ "type": "max", "fieldName": "max_friends_count", "name": "theMaxFriendsCount"},
|
||||
{ "type": "max", "fieldName": "max_follower_count", "name": "theMaxFollowerCount"},
|
||||
|
||||
{ "type": "doubleSum", "fieldName": "total_statuses_count", "name": "total_tweets_all_time"}
|
||||
|
||||
{ "type": "count", "name": "rows"},
|
||||
{ "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
|
||||
],
|
||||
"filter": { "type": "selector", "dimension": "lang", "value": "en" },
|
||||
"intervals":["2012-10-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -11,7 +11,7 @@ import com.metamx.druid.log.LogLevelAdjuster;
|
|||
import com.metamx.druid.realtime.MetadataUpdater;
|
||||
import com.metamx.druid.realtime.MetadataUpdaterConfig;
|
||||
import com.metamx.druid.realtime.RealtimeNode;
|
||||
import com.metamx.druid.realtime.SegmentPusher;
|
||||
import com.metamx.druid.loading.SegmentPusher;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
|
||||
import org.codehaus.jackson.map.jsontype.NamedType;
|
||||
|
@ -21,8 +21,6 @@ import java.io.IOException;
|
|||
|
||||
/** Standalone Demo Realtime process.
|
||||
* Created: 20121009T2050
|
||||
*
|
||||
* @author pbaclace
|
||||
*/
|
||||
public class RealtimeStandaloneMain
|
||||
{
|
||||
|
|
|
@ -1,25 +1,34 @@
|
|||
package druid.examples.twitter;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.input.InputRow;
|
||||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.realtime.Firehose;
|
||||
import com.metamx.druid.realtime.FirehoseFactory;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonTypeName;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import twitter4j.*;
|
||||
import org.codehaus.jackson.annotate.JsonTypeName;
|
||||
import twitter4j.ConnectionLifeCycleListener;
|
||||
import twitter4j.HashtagEntity;
|
||||
import twitter4j.Status;
|
||||
import twitter4j.StatusDeletionNotice;
|
||||
import twitter4j.StatusListener;
|
||||
import twitter4j.TwitterStream;
|
||||
import twitter4j.TwitterStreamFactory;
|
||||
import twitter4j.User;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.lang.Thread.*;
|
||||
import static java.lang.Thread.sleep;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -117,12 +126,8 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
|
|||
final long startMsec = System.currentTimeMillis();
|
||||
|
||||
dimensions.add("htags");
|
||||
dimensions.add("retweet_count");
|
||||
dimensions.add("follower_count");
|
||||
dimensions.add("friends_count");
|
||||
dimensions.add("lang");
|
||||
dimensions.add("utc_offset");
|
||||
dimensions.add("statuses_count");
|
||||
|
||||
//
|
||||
// set up Twitter Spritzer
|
||||
|
@ -245,39 +250,26 @@ public class TwitterSpritzerFirehoseFactory implements FirehoseFactory {
|
|||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException("InterruptedException", e);
|
||||
}
|
||||
//log.info("twitterStatus: "+ status.getCreatedAt() + " @" + status.getUser().getScreenName() + " - " + status.getText());//DEBUG
|
||||
|
||||
// theMap.put("twid", status.getUser().getScreenName());
|
||||
// theMap.put("msg", status.getText()); // ToDo: verify encoding
|
||||
|
||||
HashtagEntity[] hts = status.getHashtagEntities();
|
||||
if (hts != null && hts.length > 0) {
|
||||
// ToDo: get all the hash tags instead of just the first one
|
||||
theMap.put("htags", hts[0].getText());
|
||||
log.info("htags=" + hts[0].getText()); // about 16%
|
||||
} else {
|
||||
theMap.put("htags", null);
|
||||
List<String> hashTags = Lists.newArrayListWithExpectedSize(hts.length);
|
||||
for (HashtagEntity ht : hts) {
|
||||
hashTags.add(ht.getText());
|
||||
}
|
||||
|
||||
theMap.put("htags", Arrays.asList(hashTags.get(0)));
|
||||
}
|
||||
|
||||
long retweetCount = status.getRetweetCount();
|
||||
theMap.put("retweet_count", retweetCount);
|
||||
User u = status.getUser();
|
||||
if (u != null) {
|
||||
theMap.put("follower_count", u.getFollowersCount());
|
||||
theMap.put("friends_count", u.getFriendsCount());
|
||||
theMap.put("lang", u.getLang());
|
||||
theMap.put("utc_offset", u.getUtcOffset()); // resolution in seconds, -1 if not available?
|
||||
theMap.put("statuses_count", u.getStatusesCount());
|
||||
} else {
|
||||
log.error("status.getUser() is null");
|
||||
}
|
||||
if (rowCount % 10 == 0) {
|
||||
log.info("" + status.getCreatedAt() +
|
||||
" follower_count=" + u.getFollowersCount() +
|
||||
" friends_count=" + u.getFriendsCount() +
|
||||
" statuses_count=" + u.getStatusesCount() +
|
||||
" retweet_count=" + retweetCount
|
||||
);
|
||||
User user = status.getUser();
|
||||
if (user != null) {
|
||||
theMap.put("follower_count", user.getFollowersCount());
|
||||
theMap.put("friends_count", user.getFriendsCount());
|
||||
theMap.put("lang", user.getLang());
|
||||
theMap.put("utc_offset", user.getUtcOffset()); // resolution in seconds, -1 if not available?
|
||||
theMap.put("statuses_count", user.getStatusesCount());
|
||||
}
|
||||
|
||||
return new MapBasedInputRow(status.getCreatedAt().getTime(), dimensions, theMap);
|
||||
|
|
|
@ -41,14 +41,6 @@ druid.paths.segmentInfoCache=/tmp/twitter_realtime/segmentInfoCache
|
|||
# Path to schema definition file
|
||||
druid.request.logging.dir=/tmp/twitter_realtime/log
|
||||
|
||||
# TODO: have these moved to spec file?
|
||||
# unknown # druid.realtime.dataSources=
|
||||
# unknown # druid.realtime.index.maxSize=500000
|
||||
# unknown # druid.realtime.persistPeriod=PT600S
|
||||
# unknown # druid.realtime.scheduledExec.threads=1
|
||||
# unknown # druid.realtime.uploadPeriod=PT3600S
|
||||
# unknown # druid.realtime.windowPeriod=PT600S
|
||||
|
||||
#druid.server.maxSize=0
|
||||
druid.server.maxSize=300000000000
|
||||
# =realtime or =historical (default)
|
||||
|
|
|
@ -31,8 +31,8 @@
|
|||
|
||||
"firehose": {
|
||||
"type": "twitzer",
|
||||
"maxEventCount": 50000,
|
||||
"maxRunMinutes": 10
|
||||
"maxEventCount": 500000,
|
||||
"maxRunMinutes": 120
|
||||
},
|
||||
|
||||
"plumber": {
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.1.25-SNAPSHOT</version>
|
||||
<version>0.2.7-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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.index;
|
||||
|
||||
import com.metamx.druid.index.column.ColumnSelector;
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface QueryableIndex extends ColumnSelector
|
||||
{
|
||||
public Interval getDataInterval();
|
||||
public int getNumRows();
|
||||
public Indexed<String> getColumnNames();
|
||||
public Indexed<String> getAvailableDimensions();
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.index;
|
||||
|
||||
import com.metamx.druid.index.column.Column;
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SimpleQueryableIndex implements QueryableIndex
|
||||
{
|
||||
private final Interval dataInterval;
|
||||
private final Indexed<String> columnNames;
|
||||
private final Indexed<String> availableDimensions;
|
||||
private final Column timeColumn;
|
||||
private final Map<String, Column> otherColumns;
|
||||
|
||||
public SimpleQueryableIndex(
|
||||
Interval dataInterval,
|
||||
Indexed<String> columnNames,
|
||||
Indexed<String> dimNames,
|
||||
Column timeColumn,
|
||||
Map<String, Column> otherColumns
|
||||
)
|
||||
{
|
||||
this.dataInterval = dataInterval;
|
||||
this.columnNames = columnNames;
|
||||
this.availableDimensions = dimNames;
|
||||
this.timeColumn = timeColumn;
|
||||
this.otherColumns = otherColumns;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Interval getDataInterval()
|
||||
{
|
||||
return dataInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
return timeColumn.getLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getColumnNames()
|
||||
{
|
||||
return columnNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getAvailableDimensions()
|
||||
{
|
||||
return availableDimensions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Column getTimeColumn()
|
||||
{
|
||||
return timeColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Column getColumn(String columnName)
|
||||
{
|
||||
return otherColumns.get(columnName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class AbstractColumn implements Column
|
||||
{
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn getComplexColumn()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn getGenericColumn()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RunLengthColumn getRunLengthColumn()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DictionaryEncodedColumn getDictionaryEncoding()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface BitmapIndex
|
||||
{
|
||||
public int getCardinality();
|
||||
public String getValue(int index);
|
||||
public ImmutableConciseSet getConciseSet(String value);
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface Column
|
||||
{
|
||||
public ColumnCapabilities getCapabilities();
|
||||
|
||||
public int getLength();
|
||||
public DictionaryEncodedColumn getDictionaryEncoding();
|
||||
public RunLengthColumn getRunLengthColumn();
|
||||
public GenericColumn getGenericColumn();
|
||||
public ComplexColumn getComplexColumn();
|
||||
public BitmapIndex getBitmapIndex();
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnBuilder
|
||||
{
|
||||
private ValueType type = null;
|
||||
private boolean hasMultipleValues = false;
|
||||
|
||||
private Supplier<DictionaryEncodedColumn> dictionaryEncodedColumn = null;
|
||||
private Supplier<RunLengthColumn> runLengthColumn = null;
|
||||
private Supplier<GenericColumn> genericColumn = null;
|
||||
private Supplier<ComplexColumn> complexColumn = null;
|
||||
private Supplier<BitmapIndex> bitmapIndex = null;
|
||||
|
||||
public ColumnBuilder setType(ValueType type)
|
||||
{
|
||||
this.type = type;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setHasMultipleValues(boolean hasMultipleValues)
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setDictionaryEncodedColumn(Supplier<DictionaryEncodedColumn> dictionaryEncodedColumn)
|
||||
{
|
||||
this.dictionaryEncodedColumn = dictionaryEncodedColumn;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setRunLengthColumn(Supplier<RunLengthColumn> runLengthColumn)
|
||||
{
|
||||
this.runLengthColumn = runLengthColumn;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setGenericColumn(Supplier<GenericColumn> genericColumn)
|
||||
{
|
||||
this.genericColumn = genericColumn;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setComplexColumn(Supplier<ComplexColumn> complexColumn)
|
||||
{
|
||||
this.complexColumn = complexColumn;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setBitmapIndex(Supplier<BitmapIndex> bitmapIndex)
|
||||
{
|
||||
this.bitmapIndex = bitmapIndex;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Column build()
|
||||
{
|
||||
Preconditions.checkState(type != null, "Type must be set.");
|
||||
|
||||
return new SimpleColumn(
|
||||
new ColumnCapabilitiesImpl()
|
||||
.setType(type)
|
||||
.setDictionaryEncoded(dictionaryEncodedColumn != null)
|
||||
.setHasBitmapIndexes(bitmapIndex != null)
|
||||
.setRunLengthEncoded(runLengthColumn != null)
|
||||
.setHasMultipleValues(hasMultipleValues)
|
||||
,
|
||||
dictionaryEncodedColumn,
|
||||
runLengthColumn,
|
||||
genericColumn,
|
||||
complexColumn,
|
||||
bitmapIndex
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ColumnCapabilities
|
||||
{
|
||||
public ValueType getType();
|
||||
|
||||
public boolean isDictionaryEncoded();
|
||||
public boolean isRunLengthEncoded();
|
||||
public boolean hasBitmapIndexes();
|
||||
public boolean hasMultipleValues();
|
||||
}
|
|
@ -0,0 +1,98 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
||||
{
|
||||
private ValueType type = null;
|
||||
private boolean dictionaryEncoded = false;
|
||||
private boolean runLengthEncoded = false;
|
||||
private boolean hasInvertedIndexes = false;
|
||||
private boolean hasMultipleValues = false;
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public ValueType getType()
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setType(ValueType type)
|
||||
{
|
||||
this.type = type;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public boolean isDictionaryEncoded()
|
||||
{
|
||||
return dictionaryEncoded;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setDictionaryEncoded(boolean dictionaryEncoded)
|
||||
{
|
||||
this.dictionaryEncoded = dictionaryEncoded;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public boolean isRunLengthEncoded()
|
||||
{
|
||||
return runLengthEncoded;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setRunLengthEncoded(boolean runLengthEncoded)
|
||||
{
|
||||
this.runLengthEncoded = runLengthEncoded;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty("hasBitmapIndexes")
|
||||
public boolean hasBitmapIndexes()
|
||||
{
|
||||
return hasInvertedIndexes;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasBitmapIndexes(boolean hasInvertedIndexes)
|
||||
{
|
||||
this.hasInvertedIndexes = hasInvertedIndexes;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty("hasMultipleValues")
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return hasMultipleValues;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues)
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.druid.index.serde.ColumnPartSerde;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ColumnDescriptor
|
||||
{
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private final ValueType valueType;
|
||||
private final boolean hasMultipleValues;
|
||||
private final List<ColumnPartSerde> parts;
|
||||
|
||||
@JsonCreator
|
||||
public ColumnDescriptor(
|
||||
@JsonProperty("valueType") ValueType valueType,
|
||||
@JsonProperty("hasMultipleValues") boolean hasMultipleValues,
|
||||
@JsonProperty("parts") List<ColumnPartSerde> parts
|
||||
)
|
||||
{
|
||||
this.valueType = valueType;
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
this.parts = parts;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ValueType getValueType()
|
||||
{
|
||||
return valueType;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isHasMultipleValues()
|
||||
{
|
||||
return hasMultipleValues;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<ColumnPartSerde> getParts()
|
||||
{
|
||||
return parts;
|
||||
}
|
||||
|
||||
public int numBytes()
|
||||
{
|
||||
int retVal = 0;
|
||||
|
||||
for (ColumnPartSerde part : parts) {
|
||||
retVal += part.numBytes();
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
for (ColumnPartSerde part : parts) {
|
||||
part.write(channel);
|
||||
}
|
||||
}
|
||||
|
||||
public Column read(ByteBuffer buffer)
|
||||
{
|
||||
final ColumnBuilder builder = new ColumnBuilder()
|
||||
.setType(valueType)
|
||||
.setHasMultipleValues(hasMultipleValues);
|
||||
|
||||
for (ColumnPartSerde part : parts) {
|
||||
part.read(buffer, builder);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
private ValueType valueType = null;
|
||||
private Boolean hasMultipleValues = null;
|
||||
|
||||
private final List<ColumnPartSerde> parts = Lists.newArrayList();
|
||||
|
||||
public Builder setValueType(ValueType valueType)
|
||||
{
|
||||
if (this.valueType != null && this.valueType != valueType) {
|
||||
throw new IAE("valueType[%s] is already set, cannot change to[%s]", this.valueType, valueType);
|
||||
}
|
||||
this.valueType = valueType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setHasMultipleValues(boolean hasMultipleValues)
|
||||
{
|
||||
if (this.hasMultipleValues != null && this.hasMultipleValues != hasMultipleValues) {
|
||||
throw new IAE(
|
||||
"hasMultipleValues[%s] is already set, cannot change to[%s]", this.hasMultipleValues, hasMultipleValues
|
||||
);
|
||||
}
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder addSerde(ColumnPartSerde serde)
|
||||
{
|
||||
parts.add(serde);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ColumnDescriptor build()
|
||||
{
|
||||
Preconditions.checkNotNull(valueType, "must specify a valueType");
|
||||
return new ColumnDescriptor(valueType, hasMultipleValues == null ? false : hasMultipleValues, parts);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,12 +17,12 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.client.cache;
|
||||
package com.metamx.druid.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface CacheBroker
|
||||
public interface ColumnSelector
|
||||
{
|
||||
public CacheStats getStats();
|
||||
public Cache provideCache(String identifier);
|
||||
public Column getTimeColumn();
|
||||
public Column getColumn(String columnName);
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ComplexColumn extends Closeable
|
||||
{
|
||||
public Class<?> getClazz();
|
||||
public String getTypeName();
|
||||
public Object getRowValue(int rowNum);
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ComplexColumnImpl extends AbstractColumn
|
||||
{
|
||||
private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.COMPLEX);
|
||||
|
||||
private final Indexed column;
|
||||
private final String typeName;
|
||||
|
||||
public ComplexColumnImpl(String typeName, Indexed column)
|
||||
{
|
||||
this.column = column;
|
||||
this.typeName = typeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return CAPABILITIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn getComplexColumn()
|
||||
{
|
||||
return new IndexedComplexColumn(typeName, column);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.IndexedInts;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface DictionaryEncodedColumn
|
||||
{
|
||||
public int length();
|
||||
public boolean hasMultipleValues();
|
||||
public int getSingleValueRow(int rowNum);
|
||||
public IndexedInts getMultiValueRow(int rowNum);
|
||||
public String lookupName(int id);
|
||||
public int lookupId(String name);
|
||||
public int getCardinality();
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FloatColumn extends AbstractColumn
|
||||
{
|
||||
private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.FLOAT);
|
||||
|
||||
private final CompressedFloatsIndexedSupplier column;
|
||||
|
||||
public FloatColumn(CompressedFloatsIndexedSupplier column)
|
||||
{
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return CAPABILITIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn getGenericColumn()
|
||||
{
|
||||
return new IndexedFloatsGenericColumn(column.get());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import com.metamx.druid.kv.IndexedFloats;
|
||||
import com.metamx.druid.kv.IndexedLongs;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface GenericColumn extends Closeable
|
||||
{
|
||||
public int length();
|
||||
public ValueType getType();
|
||||
public boolean hasMultipleValues();
|
||||
|
||||
public String getStringSingleValueRow(int rowNum);
|
||||
public Indexed<String> getStringMultiValueRow(int rowNum);
|
||||
public float getFloatSingleValueRow(int rowNum);
|
||||
public IndexedFloats getFloatMultiValueRow(int rowNum);
|
||||
public long getLongSingleValueRow(int rowNum);
|
||||
public IndexedLongs getLongMultiValueRow(int rowNum);
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedComplexColumn implements ComplexColumn
|
||||
{
|
||||
private final Indexed column;
|
||||
private final String typeName;
|
||||
|
||||
public IndexedComplexColumn(
|
||||
String typeName, Indexed column
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
this.typeName = typeName;
|
||||
}
|
||||
@Override
|
||||
public Class<?> getClazz()
|
||||
{
|
||||
return column.getClazz();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTypeName()
|
||||
{
|
||||
return typeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRowValue(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import com.metamx.druid.kv.IndexedFloats;
|
||||
import com.metamx.druid.kv.IndexedLongs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedFloatsGenericColumn implements GenericColumn
|
||||
{
|
||||
private final IndexedFloats column;
|
||||
|
||||
public IndexedFloatsGenericColumn(
|
||||
final IndexedFloats column
|
||||
) {
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType getType()
|
||||
{
|
||||
return ValueType.FLOAT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStringSingleValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getStringMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloatSingleValueRow(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedFloats getFloatMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLongSingleValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedLongs getLongMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
column.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import com.metamx.druid.kv.IndexedFloats;
|
||||
import com.metamx.druid.kv.IndexedLongs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexedLongsGenericColumn implements GenericColumn
|
||||
{
|
||||
private final IndexedLongs column;
|
||||
|
||||
public IndexedLongsGenericColumn(
|
||||
final IndexedLongs column
|
||||
) {
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueType getType()
|
||||
{
|
||||
return ValueType.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStringSingleValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getStringMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloatSingleValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedFloats getFloatMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLongSingleValueRow(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedLongs getLongMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
column.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongColumn extends AbstractColumn
|
||||
{
|
||||
private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.LONG);
|
||||
|
||||
private final CompressedLongsIndexedSupplier column;
|
||||
|
||||
public LongColumn(CompressedLongsIndexedSupplier column)
|
||||
{
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return CAPABILITIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn getGenericColumn()
|
||||
{
|
||||
return new IndexedLongsGenericColumn(column.get());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface RunLengthColumn
|
||||
{
|
||||
public void thisIsAFictionalInterfaceThatWillHopefullyMeanSomethingSometime();
|
||||
}
|
|
@ -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.index.column;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.io.Closeables;
|
||||
|
||||
/**
|
||||
*/
|
||||
class SimpleColumn implements Column
|
||||
{
|
||||
private final ColumnCapabilities capabilities;
|
||||
private final Supplier<DictionaryEncodedColumn> dictionaryEncodedColumn;
|
||||
private final Supplier<RunLengthColumn> runLengthColumn;
|
||||
private final Supplier<GenericColumn> genericColumn;
|
||||
private final Supplier<ComplexColumn> complexColumn;
|
||||
private final Supplier<BitmapIndex> bitmapIndex;
|
||||
|
||||
SimpleColumn(
|
||||
ColumnCapabilities capabilities,
|
||||
Supplier<DictionaryEncodedColumn> dictionaryEncodedColumn,
|
||||
Supplier<RunLengthColumn> runLengthColumn,
|
||||
Supplier<GenericColumn> genericColumn,
|
||||
Supplier<ComplexColumn> complexColumn,
|
||||
Supplier<BitmapIndex> bitmapIndex
|
||||
)
|
||||
{
|
||||
this.capabilities = capabilities;
|
||||
this.dictionaryEncodedColumn = dictionaryEncodedColumn;
|
||||
this.runLengthColumn = runLengthColumn;
|
||||
this.genericColumn = genericColumn;
|
||||
this.complexColumn = complexColumn;
|
||||
this.bitmapIndex = bitmapIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return capabilities;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
GenericColumn column = null;
|
||||
try {
|
||||
column = genericColumn.get();
|
||||
return column.length();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(column);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DictionaryEncodedColumn getDictionaryEncoding()
|
||||
{
|
||||
return dictionaryEncodedColumn.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RunLengthColumn getRunLengthColumn()
|
||||
{
|
||||
return runLengthColumn.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn getGenericColumn()
|
||||
{
|
||||
return genericColumn.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn getComplexColumn()
|
||||
{
|
||||
return complexColumn.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex()
|
||||
{
|
||||
return bitmapIndex.get();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedInts;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn
|
||||
{
|
||||
private final VSizeIndexedInts column;
|
||||
private final VSizeIndexed multiValueColumn;
|
||||
private final GenericIndexed<String> lookups;
|
||||
|
||||
public SimpleDictionaryEncodedColumn(
|
||||
VSizeIndexedInts singleValueColumn,
|
||||
VSizeIndexed multiValueColumn,
|
||||
GenericIndexed<String> lookups
|
||||
)
|
||||
{
|
||||
this.column = singleValueColumn;
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.lookups = lookups;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return hasMultipleValues() ? multiValueColumn.size() : column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return column == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSingleValueRow(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getMultiValueRow(int rowNum)
|
||||
{
|
||||
return multiValueColumn.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return lookups.get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return lookups.indexOf(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return lookups.size();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,118 @@
|
|||
/*
|
||||
* 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.index.column;
|
||||
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import com.metamx.druid.kv.IndexedInts;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class StringMultiValueColumn extends AbstractColumn
|
||||
{
|
||||
private static final ImmutableConciseSet emptySet = new ImmutableConciseSet();
|
||||
private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl()
|
||||
.setType(ValueType.STRING)
|
||||
.setDictionaryEncoded(true)
|
||||
.setHasBitmapIndexes(true)
|
||||
.setHasMultipleValues(true);
|
||||
|
||||
private final Indexed<String> lookups;
|
||||
private final Indexed<? extends IndexedInts> column;
|
||||
private final Indexed<ImmutableConciseSet> bitmapIndexes;
|
||||
|
||||
public StringMultiValueColumn(
|
||||
Indexed<String> lookups,
|
||||
Indexed<? extends IndexedInts> column,
|
||||
Indexed<ImmutableConciseSet> bitmapIndexes
|
||||
)
|
||||
{
|
||||
this.lookups = lookups;
|
||||
this.column = column;
|
||||
this.bitmapIndexes = bitmapIndexes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnCapabilities getCapabilities()
|
||||
{
|
||||
return CAPABILITIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DictionaryEncodedColumn getDictionaryEncoding()
|
||||
{
|
||||
return new DictionaryEncodedColumn()
|
||||
{
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSingleValueRow(int rowNum)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getMultiValueRow(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return lookups.get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return lookups.indexOf(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return lookups.size();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,11 @@
|
|||
package com.metamx.druid.index.column;
|
||||
|
||||
/**
|
||||
*/
|
||||
public enum ValueType
|
||||
{
|
||||
FLOAT,
|
||||
LONG,
|
||||
STRING,
|
||||
COMPLEX
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.druid.index.column.BitmapIndex;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.Indexed;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex>
|
||||
{
|
||||
private static final ImmutableConciseSet EMPTY_SET = new ImmutableConciseSet();
|
||||
|
||||
private final GenericIndexed<ImmutableConciseSet> bitmaps;
|
||||
private final GenericIndexed<String> dictionary;
|
||||
|
||||
public BitmapIndexColumnPartSupplier(
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps,
|
||||
GenericIndexed<String> dictionary
|
||||
) {
|
||||
this.bitmaps = bitmaps;
|
||||
this.dictionary = dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex get()
|
||||
{
|
||||
return new BitmapIndex()
|
||||
{
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return dictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getValue(int index)
|
||||
{
|
||||
return dictionary.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableConciseSet getConciseSet(String value)
|
||||
{
|
||||
final int index = dictionary.indexOf(value);
|
||||
|
||||
if (index < 0) {
|
||||
return EMPTY_SET;
|
||||
}
|
||||
|
||||
final ImmutableConciseSet bitmap = bitmaps.get(index);
|
||||
return bitmap == null ? EMPTY_SET : bitmap;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.index.serde;
|
||||
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import org.codehaus.jackson.annotate.JsonSubTypes;
|
||||
import org.codehaus.jackson.annotate.JsonTypeInfo;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "complex", value = ComplexColumnPartSerde.class),
|
||||
@JsonSubTypes.Type(name = "float", value = FloatGenericColumnPartSerde.class),
|
||||
@JsonSubTypes.Type(name = "long", value = LongGenericColumnPartSerde.class),
|
||||
@JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class)
|
||||
})
|
||||
public interface ColumnPartSerde
|
||||
{
|
||||
public int numBytes();
|
||||
public void write(WritableByteChannel channel) throws IOException;
|
||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder);
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.v1.serde.ComplexMetricSerde;
|
||||
import com.metamx.druid.index.v1.serde.ComplexMetrics;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ComplexColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
public static ComplexColumnPartSerde createDeserializer(
|
||||
@JsonProperty("typeName") String complexType
|
||||
)
|
||||
{
|
||||
return new ComplexColumnPartSerde(null, complexType);
|
||||
}
|
||||
|
||||
private final GenericIndexed column;
|
||||
private final String typeName;
|
||||
|
||||
private final ComplexMetricSerde serde;
|
||||
|
||||
public ComplexColumnPartSerde(GenericIndexed column, String typeName)
|
||||
{
|
||||
this.column = column;
|
||||
this.typeName = typeName;
|
||||
serde = ComplexMetrics.getSerdeForType(typeName);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getTypeName()
|
||||
{
|
||||
return typeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numBytes()
|
||||
{
|
||||
return column.getSerializedSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
column.writeToChannel(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
||||
{
|
||||
return serde.deserializeColumn(buffer, builder);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.druid.index.column.ComplexColumn;
|
||||
import com.metamx.druid.index.column.IndexedComplexColumn;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ComplexColumnPartSupplier implements Supplier<ComplexColumn>
|
||||
{
|
||||
private final GenericIndexed complexType;
|
||||
private final String typeName;
|
||||
|
||||
public ComplexColumnPartSupplier(
|
||||
final String typeName, final GenericIndexed complexType
|
||||
) {
|
||||
this.complexType = complexType;
|
||||
this.typeName = typeName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexColumn get()
|
||||
{
|
||||
return new IndexedComplexColumn(typeName, complexType);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
public static DictionaryEncodedColumnPartSerde createDeserializer(boolean singleValued)
|
||||
{
|
||||
return new DictionaryEncodedColumnPartSerde();
|
||||
}
|
||||
|
||||
private final GenericIndexed<String> dictionary;
|
||||
private final VSizeIndexedInts singleValuedColumn;
|
||||
private final VSizeIndexed multiValuedColumn;
|
||||
private final GenericIndexed<ImmutableConciseSet> bitmaps;
|
||||
|
||||
private final int size;
|
||||
|
||||
public DictionaryEncodedColumnPartSerde(
|
||||
GenericIndexed<String> dictionary,
|
||||
VSizeIndexedInts singleValCol,
|
||||
VSizeIndexed multiValCol,
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps
|
||||
)
|
||||
{
|
||||
this.dictionary = dictionary;
|
||||
this.singleValuedColumn = singleValCol;
|
||||
this.multiValuedColumn = multiValCol;
|
||||
this.bitmaps = bitmaps;
|
||||
|
||||
int size = dictionary.getSerializedSize();
|
||||
if (singleValCol != null && multiValCol == null) {
|
||||
size += singleValCol.getSerializedSize();
|
||||
}
|
||||
else if (singleValCol == null && multiValCol != null) {
|
||||
size += multiValCol.getSerializedSize();
|
||||
}
|
||||
else {
|
||||
throw new IAE("Either singleValCol[%s] or multiValCol[%s] must be set", singleValCol, multiValCol);
|
||||
}
|
||||
size += bitmaps.getSerializedSize();
|
||||
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
private DictionaryEncodedColumnPartSerde()
|
||||
{
|
||||
dictionary = null;
|
||||
singleValuedColumn = null;
|
||||
multiValuedColumn = null;
|
||||
bitmaps = null;
|
||||
size = 0;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
private boolean isSingleValued()
|
||||
{
|
||||
return singleValuedColumn != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numBytes()
|
||||
{
|
||||
return 1 + size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(new byte[]{(byte) (isSingleValued() ? 0x0 : 0x1)}));
|
||||
dictionary.writeToChannel(channel);
|
||||
if (isSingleValued()) {
|
||||
singleValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
else {
|
||||
multiValuedColumn.writeToChannel(channel);
|
||||
}
|
||||
bitmaps.writeToChannel(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
||||
{
|
||||
final boolean isSingleValued = buffer.get() == 0x0;
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy);
|
||||
final VSizeIndexedInts singleValuedColumn;
|
||||
final VSizeIndexed multiValuedColumn;
|
||||
|
||||
builder.setType(ValueType.STRING);
|
||||
|
||||
if (isSingleValued) {
|
||||
singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer);
|
||||
multiValuedColumn = null;
|
||||
builder.setHasMultipleValues(false)
|
||||
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null));
|
||||
}
|
||||
else {
|
||||
singleValuedColumn = null;
|
||||
multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer);
|
||||
builder.setHasMultipleValues(true)
|
||||
.setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn));
|
||||
}
|
||||
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps = GenericIndexed.read(
|
||||
buffer, ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
|
||||
builder.setBitmapIndex(new BitmapIndexColumnPartSupplier(bitmaps, dictionary));
|
||||
|
||||
return new DictionaryEncodedColumnPartSerde(dictionary, singleValuedColumn, multiValuedColumn, bitmaps);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.druid.index.column.DictionaryEncodedColumn;
|
||||
import com.metamx.druid.index.column.SimpleDictionaryEncodedColumn;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn>
|
||||
{
|
||||
private final GenericIndexed<String> dictionary;
|
||||
private final VSizeIndexedInts singleValuedColumn;
|
||||
private final VSizeIndexed multiValuedColumn;
|
||||
|
||||
public DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed<String> dictionary,
|
||||
VSizeIndexedInts singleValuedColumn,
|
||||
VSizeIndexed multiValuedColumn
|
||||
)
|
||||
{
|
||||
this.dictionary = dictionary;
|
||||
this.singleValuedColumn = singleValuedColumn;
|
||||
this.multiValuedColumn = multiValuedColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DictionaryEncodedColumn get()
|
||||
{
|
||||
return new SimpleDictionaryEncodedColumn(singleValuedColumn, multiValuedColumn, dictionary);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FloatGenericColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
public static FloatGenericColumnPartSerde createDeserializer(
|
||||
@JsonProperty("byteOrder") ByteOrder byteOrder
|
||||
)
|
||||
{
|
||||
return new FloatGenericColumnPartSerde(null, byteOrder);
|
||||
}
|
||||
|
||||
private final CompressedFloatsIndexedSupplier compressedFloats;
|
||||
private final ByteOrder byteOrder;
|
||||
|
||||
public FloatGenericColumnPartSerde(CompressedFloatsIndexedSupplier compressedFloats, ByteOrder byteOrder)
|
||||
{
|
||||
this.compressedFloats = compressedFloats;
|
||||
this.byteOrder = byteOrder;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ByteOrder getByteOrder()
|
||||
{
|
||||
return byteOrder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numBytes()
|
||||
{
|
||||
return compressedFloats.getSerializedSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
compressedFloats.writeToChannel(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
||||
{
|
||||
final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
|
||||
|
||||
builder.setType(ValueType.FLOAT)
|
||||
.setHasMultipleValues(false)
|
||||
.setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder));
|
||||
|
||||
return new FloatGenericColumnPartSerde(column, byteOrder);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.druid.index.column.GenericColumn;
|
||||
import com.metamx.druid.index.column.IndexedFloatsGenericColumn;
|
||||
import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier;
|
||||
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FloatGenericColumnSupplier implements Supplier<GenericColumn>
|
||||
{
|
||||
private final CompressedFloatsIndexedSupplier column;
|
||||
private final ByteOrder byteOrder;
|
||||
|
||||
public FloatGenericColumnSupplier(
|
||||
CompressedFloatsIndexedSupplier column,
|
||||
ByteOrder byteOrder
|
||||
) {
|
||||
this.column = column;
|
||||
this.byteOrder = byteOrder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn get()
|
||||
{
|
||||
return new IndexedFloatsGenericColumn(column.get());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier;
|
||||
import org.codehaus.jackson.annotate.JsonCreator;
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongGenericColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
@JsonCreator
|
||||
public static LongGenericColumnPartSerde createDeserializer(
|
||||
@JsonProperty("byteOrder") ByteOrder byteOrder
|
||||
)
|
||||
{
|
||||
return new LongGenericColumnPartSerde(null, byteOrder);
|
||||
}
|
||||
|
||||
private final CompressedLongsIndexedSupplier compressedLongs;
|
||||
private final ByteOrder byteOrder;
|
||||
|
||||
public LongGenericColumnPartSerde(CompressedLongsIndexedSupplier compressedLongs, ByteOrder byteOrder)
|
||||
{
|
||||
this.compressedLongs = compressedLongs;
|
||||
this.byteOrder = byteOrder;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ByteOrder getByteOrder()
|
||||
{
|
||||
return byteOrder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numBytes()
|
||||
{
|
||||
return compressedLongs.getSerializedSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
compressedLongs.writeToChannel(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder)
|
||||
{
|
||||
final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder);
|
||||
|
||||
builder.setType(ValueType.LONG)
|
||||
.setHasMultipleValues(false)
|
||||
.setGenericColumn(new LongGenericColumnSupplier(column));
|
||||
|
||||
return new LongGenericColumnPartSerde(column, byteOrder);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* 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.index.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.metamx.druid.index.column.GenericColumn;
|
||||
import com.metamx.druid.index.column.IndexedLongsGenericColumn;
|
||||
import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier;
|
||||
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongGenericColumnSupplier implements Supplier<GenericColumn>
|
||||
{
|
||||
private final CompressedLongsIndexedSupplier column;
|
||||
|
||||
public LongGenericColumnSupplier(
|
||||
CompressedLongsIndexedSupplier column
|
||||
) {
|
||||
this.column = column;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumn get()
|
||||
{
|
||||
return new IndexedLongsGenericColumn(column.get());
|
||||
}
|
||||
}
|
|
@ -29,7 +29,6 @@ import com.metamx.druid.collect.ResourceHolder;
|
|||
import com.metamx.druid.collect.StupidResourceHolder;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedFloats;
|
||||
import com.metamx.druid.kv.IndexedIterable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -60,6 +59,11 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
|
|||
this.baseFloatBuffers = baseFloatBuffers;
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
return totalSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedFloats get()
|
||||
{
|
||||
|
@ -146,6 +150,11 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
|
|||
};
|
||||
}
|
||||
|
||||
public int getSerializedSize()
|
||||
{
|
||||
return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4;
|
||||
}
|
||||
|
||||
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(new byte[]{version}));
|
||||
|
@ -178,7 +187,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
|
|||
return MAX_FLOATS_IN_BUFFER - (MAX_FLOATS_IN_BUFFER % numFloatsInChunk);
|
||||
}
|
||||
|
||||
public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) throws IOException
|
||||
public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order)
|
||||
{
|
||||
byte versionFromBuffer = buffer.get();
|
||||
|
||||
|
@ -186,7 +195,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier<IndexedFloats>
|
|||
return new CompressedFloatsIndexedSupplier(
|
||||
buffer.getInt(),
|
||||
buffer.getInt(),
|
||||
GenericIndexed.readFromByteBuffer(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order))
|
||||
GenericIndexed.read(buffer, CompressedFloatBufferObjectStrategy.getBufferForOrder(order))
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,6 @@ import com.metamx.common.IAE;
|
|||
import com.metamx.druid.collect.ResourceHolder;
|
||||
import com.metamx.druid.collect.StupidResourceHolder;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedIterable;
|
||||
import com.metamx.druid.kv.IndexedLongs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -162,6 +161,11 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
|||
};
|
||||
}
|
||||
|
||||
public int getSerializedSize()
|
||||
{
|
||||
return baseLongBuffers.getSerializedSize() + 1 + 4 + 4;
|
||||
}
|
||||
|
||||
public void writeToChannel(WritableByteChannel channel) throws IOException
|
||||
{
|
||||
channel.write(ByteBuffer.wrap(new byte[]{version}));
|
||||
|
@ -188,7 +192,7 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
|||
return baseLongBuffers;
|
||||
}
|
||||
|
||||
public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) throws IOException
|
||||
public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order)
|
||||
{
|
||||
byte versionFromBuffer = buffer.get();
|
||||
|
||||
|
@ -196,7 +200,7 @@ public class CompressedLongsIndexedSupplier implements Supplier<IndexedLongs>
|
|||
return new CompressedLongsIndexedSupplier(
|
||||
buffer.getInt(),
|
||||
buffer.getInt(),
|
||||
GenericIndexed.readFromByteBuffer(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order))
|
||||
GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order))
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -524,7 +524,7 @@ public class IncrementalIndex implements Iterable<Row>
|
|||
|
||||
public String get(String value)
|
||||
{
|
||||
return poorMansInterning.get(value);
|
||||
return value == null ? null : poorMansInterning.get(value);
|
||||
}
|
||||
|
||||
public int getId(String value)
|
||||
|
|
|
@ -20,35 +20,96 @@
|
|||
package com.metamx.druid.index.v1;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.common.IAE;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.io.smoosh.FileSmoosher;
|
||||
import com.metamx.common.io.smoosh.Smoosh;
|
||||
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||
import com.metamx.common.io.smoosh.SmooshedWriter;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.index.QueryableIndex;
|
||||
import com.metamx.druid.index.SimpleQueryableIndex;
|
||||
import com.metamx.druid.index.column.Column;
|
||||
import com.metamx.druid.index.column.ColumnBuilder;
|
||||
import com.metamx.druid.index.column.ColumnDescriptor;
|
||||
import com.metamx.druid.index.column.ValueType;
|
||||
import com.metamx.druid.index.serde.BitmapIndexColumnPartSupplier;
|
||||
import com.metamx.druid.index.serde.ComplexColumnPartSerde;
|
||||
import com.metamx.druid.index.serde.ComplexColumnPartSupplier;
|
||||
import com.metamx.druid.index.serde.DictionaryEncodedColumnPartSerde;
|
||||
import com.metamx.druid.index.serde.DictionaryEncodedColumnSupplier;
|
||||
import com.metamx.druid.index.serde.FloatGenericColumnPartSerde;
|
||||
import com.metamx.druid.index.serde.FloatGenericColumnSupplier;
|
||||
import com.metamx.druid.index.serde.LongGenericColumnPartSerde;
|
||||
import com.metamx.druid.index.serde.LongGenericColumnSupplier;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.kv.ArrayIndexed;
|
||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||
import com.metamx.druid.kv.GenericIndexed;
|
||||
import com.metamx.druid.kv.IndexedIterable;
|
||||
import com.metamx.druid.kv.VSizeIndexed;
|
||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||
import com.metamx.druid.utils.SerializerUtils;
|
||||
import it.uniroma3.mat.extendedset.intset.ConciseSet;
|
||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.AbstractList;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This should be changed to use DI instead of a static reference...
|
||||
*/
|
||||
public class IndexIO
|
||||
{
|
||||
public static final byte V8_VERSION = 0x8;
|
||||
public static final byte V9_VERSION = 0x9;
|
||||
|
||||
private static final Map<Integer, IndexLoader> indexLoaders =
|
||||
ImmutableMap.<Integer, IndexLoader>builder()
|
||||
.put(0, new LegacyIndexLoader())
|
||||
.put(1, new LegacyIndexLoader())
|
||||
.put(2, new LegacyIndexLoader())
|
||||
.put(3, new LegacyIndexLoader())
|
||||
.put(4, new LegacyIndexLoader())
|
||||
.put(5, new LegacyIndexLoader())
|
||||
.put(6, new LegacyIndexLoader())
|
||||
.put(7, new LegacyIndexLoader())
|
||||
.put(8, new LegacyIndexLoader())
|
||||
.put(9, new V9IndexLoader())
|
||||
.build();
|
||||
|
||||
private static final Logger log = new Logger(IndexIO.class);
|
||||
private static final SerializerUtils serializerUtils = new SerializerUtils();
|
||||
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
|
||||
private static final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
|
||||
private static volatile IndexIOHandler handler = null;
|
||||
public static final byte CURRENT_VERSION_ID = 0x8;
|
||||
public static final int CURRENT_VERSION_ID = V9_VERSION;
|
||||
|
||||
public static Index readIndex(File inDir) throws IOException
|
||||
{
|
||||
|
@ -62,12 +123,27 @@ public class IndexIO
|
|||
return handler.canBeMapped(inDir);
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public static MMappedIndex mapDir(final File inDir) throws IOException
|
||||
{
|
||||
init();
|
||||
return handler.mapDir(inDir);
|
||||
}
|
||||
|
||||
public static QueryableIndex loadIndex(File inDir) throws IOException
|
||||
{
|
||||
init();
|
||||
final int version = getVersionFromDir(inDir);
|
||||
|
||||
final IndexLoader loader = indexLoaders.get(version);
|
||||
|
||||
if (loader != null) {
|
||||
return loader.load(inDir);
|
||||
} else {
|
||||
throw new ISE("Unknown index version[%s]", version);
|
||||
}
|
||||
}
|
||||
|
||||
public static void storeLatest(Index index, File file) throws IOException
|
||||
{
|
||||
handler.storeLatest(index, file);
|
||||
|
@ -82,8 +158,7 @@ public class IndexIO
|
|||
{
|
||||
if (IndexIO.handler == null) {
|
||||
IndexIO.handler = handler;
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
throw new ISE("Already have a handler[%s], cannot register another[%s]", IndexIO.handler, handler);
|
||||
}
|
||||
}
|
||||
|
@ -95,6 +170,26 @@ public class IndexIO
|
|||
}
|
||||
}
|
||||
|
||||
public static int getVersionFromDir(File inDir) throws IOException
|
||||
{
|
||||
File versionFile = new File(inDir, "version.bin");
|
||||
if (versionFile.exists()) {
|
||||
return Ints.fromByteArray(Files.toByteArray(versionFile));
|
||||
}
|
||||
|
||||
final File indexFile = new File(inDir, "index.drd");
|
||||
InputStream in = null;
|
||||
int version;
|
||||
try {
|
||||
in = new FileInputStream(indexFile);
|
||||
version = in.read();
|
||||
}
|
||||
finally {
|
||||
Closeables.closeQuietly(in);
|
||||
}
|
||||
return version;
|
||||
}
|
||||
|
||||
public static void checkFileSize(File indexFile) throws IOException
|
||||
{
|
||||
final long fileSize = indexFile.length();
|
||||
|
@ -128,7 +223,6 @@ public class IndexIO
|
|||
* This only exists for some legacy compatibility reasons, Metamarkets is working on getting rid of it in
|
||||
* future versions. Normal persisting of indexes is done via IndexMerger.
|
||||
*
|
||||
*
|
||||
* @param file
|
||||
*/
|
||||
public void storeLatest(Index index, File file) throws IOException;
|
||||
|
@ -137,9 +231,6 @@ public class IndexIO
|
|||
static class DefaultIndexIOHandler implements IndexIOHandler
|
||||
{
|
||||
private static final Logger log = new Logger(DefaultIndexIOHandler.class);
|
||||
private static final SerializerUtils serializerUtils = new SerializerUtils();
|
||||
private static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
|
||||
|
||||
@Override
|
||||
public Index readIndex(File inDir)
|
||||
{
|
||||
|
@ -152,8 +243,6 @@ public class IndexIO
|
|||
return true;
|
||||
}
|
||||
|
||||
public static final byte VERSION_ID = 0x8;
|
||||
|
||||
@Override
|
||||
public MMappedIndex mapDir(File inDir) throws IOException
|
||||
{
|
||||
|
@ -164,7 +253,7 @@ public class IndexIO
|
|||
try {
|
||||
indexIn = new FileInputStream(new File(inDir, "index.drd"));
|
||||
byte theVersion = (byte) indexIn.read();
|
||||
if (theVersion != VERSION_ID) {
|
||||
if (theVersion != V8_VERSION) {
|
||||
throw new IllegalArgumentException(String.format("Unknown version[%s]", theVersion));
|
||||
}
|
||||
}
|
||||
|
@ -176,10 +265,10 @@ public class IndexIO
|
|||
ByteBuffer indexBuffer = smooshedFiles.mapFile("index.drd");
|
||||
|
||||
indexBuffer.get(); // Skip the version byte
|
||||
final GenericIndexed<String> availableDimensions = GenericIndexed.readFromByteBuffer(
|
||||
final GenericIndexed<String> availableDimensions = GenericIndexed.read(
|
||||
indexBuffer, GenericIndexed.stringStrategy
|
||||
);
|
||||
final GenericIndexed<String> availableMetrics = GenericIndexed.readFromByteBuffer(
|
||||
final GenericIndexed<String> availableMetrics = GenericIndexed.read(
|
||||
indexBuffer, GenericIndexed.stringStrategy
|
||||
);
|
||||
final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer));
|
||||
|
@ -213,7 +302,7 @@ public class IndexIO
|
|||
fileDimensionName
|
||||
);
|
||||
|
||||
dimValueLookups.put(dimension, GenericIndexed.readFromByteBuffer(dimBuffer, GenericIndexed.stringStrategy));
|
||||
dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.stringStrategy));
|
||||
dimColumns.put(dimension, VSizeIndexed.readFromByteBuffer(dimBuffer));
|
||||
}
|
||||
|
||||
|
@ -221,7 +310,7 @@ public class IndexIO
|
|||
for (int i = 0; i < availableDimensions.size(); ++i) {
|
||||
invertedIndexed.put(
|
||||
serializerUtils.readString(invertedBuffer),
|
||||
GenericIndexed.readFromByteBuffer(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy)
|
||||
GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -246,6 +335,386 @@ public class IndexIO
|
|||
{
|
||||
throw new UnsupportedOperationException("Shouldn't ever happen in a cluster that is not owned by MMX.");
|
||||
}
|
||||
|
||||
public static void convertV8toV9(File v8Dir, File v9Dir) throws IOException
|
||||
{
|
||||
log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir);
|
||||
|
||||
InputStream indexIn = null;
|
||||
try {
|
||||
indexIn = new FileInputStream(new File(v8Dir, "index.drd"));
|
||||
byte theVersion = (byte) indexIn.read();
|
||||
if (theVersion != V8_VERSION) {
|
||||
throw new IAE("Unknown version[%s]", theVersion);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
Closeables.close(indexIn, false);
|
||||
}
|
||||
|
||||
SmooshedFileMapper v8SmooshedFiles = Smoosh.map(v8Dir);
|
||||
|
||||
v9Dir.mkdirs();
|
||||
final FileSmoosher v9Smoosher = new FileSmoosher(v9Dir);
|
||||
|
||||
ByteStreams.write(Ints.toByteArray(9), Files.newOutputStreamSupplier(new File(v9Dir, "version.bin")));
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> bitmapIndexes = Maps.newHashMap();
|
||||
|
||||
final ByteBuffer invertedBuffer = v8SmooshedFiles.mapFile("inverted.drd");
|
||||
while (invertedBuffer.hasRemaining()) {
|
||||
bitmapIndexes.put(
|
||||
serializerUtils.readString(invertedBuffer),
|
||||
GenericIndexed.read(invertedBuffer, ConciseCompressedIndexedInts.objectStrategy)
|
||||
);
|
||||
}
|
||||
|
||||
LinkedHashSet<String> skippedFiles = Sets.newLinkedHashSet();
|
||||
for (String filename : v8SmooshedFiles.getInternalFilenames()) {
|
||||
log.info("Processing file[%s]", filename);
|
||||
if (filename.startsWith("dim_")) {
|
||||
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
|
||||
builder.setValueType(ValueType.STRING);
|
||||
|
||||
final List<ByteBuffer> outParts = Lists.newArrayList();
|
||||
|
||||
ByteBuffer dimBuffer = v8SmooshedFiles.mapFile(filename);
|
||||
String dimension = serializerUtils.readString(dimBuffer);
|
||||
if (!filename.equals(String.format("dim_%s.drd", dimension))) {
|
||||
throw new ISE("loaded dimension[%s] from file[%s]", dimension, filename);
|
||||
}
|
||||
|
||||
ByteArrayOutputStream nameBAOS = new ByteArrayOutputStream();
|
||||
serializerUtils.writeString(nameBAOS, dimension);
|
||||
outParts.add(ByteBuffer.wrap(nameBAOS.toByteArray()));
|
||||
|
||||
GenericIndexed<String> dictionary = GenericIndexed.read(
|
||||
dimBuffer, GenericIndexed.stringStrategy
|
||||
);
|
||||
|
||||
VSizeIndexedInts singleValCol = null;
|
||||
VSizeIndexed multiValCol = VSizeIndexed.readFromByteBuffer(dimBuffer.asReadOnlyBuffer());
|
||||
GenericIndexed<ImmutableConciseSet> bitmaps = bitmapIndexes.get(dimension);
|
||||
|
||||
boolean onlyOneValue = true;
|
||||
ConciseSet nullsSet = null;
|
||||
for (int i = 0; i < multiValCol.size(); ++i) {
|
||||
VSizeIndexedInts rowValue = multiValCol.get(i);
|
||||
if (!onlyOneValue) {
|
||||
break;
|
||||
}
|
||||
if (rowValue.size() > 1) {
|
||||
onlyOneValue = false;
|
||||
}
|
||||
if (rowValue.size() == 0) {
|
||||
if (nullsSet == null) {
|
||||
nullsSet = new ConciseSet();
|
||||
}
|
||||
nullsSet.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
if (onlyOneValue) {
|
||||
log.info("Dimension[%s] is single value, converting...", dimension);
|
||||
final boolean bumpedDictionary;
|
||||
if (nullsSet != null) {
|
||||
log.info("Dimension[%s] has null rows.", dimension);
|
||||
final ImmutableConciseSet theNullSet = ImmutableConciseSet.newImmutableFromMutable(nullsSet);
|
||||
|
||||
if (dictionary.get(0) != null) {
|
||||
log.info("Dimension[%s] has no null value in the dictionary, expanding...", dimension);
|
||||
bumpedDictionary = true;
|
||||
final List<String> nullList = Lists.newArrayList();
|
||||
nullList.add(null);
|
||||
|
||||
dictionary = GenericIndexed.fromIterable(
|
||||
Iterables.concat(nullList, dictionary),
|
||||
GenericIndexed.stringStrategy
|
||||
);
|
||||
|
||||
bitmaps = GenericIndexed.fromIterable(
|
||||
Iterables.concat(Arrays.asList(theNullSet), bitmaps),
|
||||
ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
}
|
||||
else {
|
||||
bumpedDictionary = false;
|
||||
bitmaps = GenericIndexed.fromIterable(
|
||||
Iterables.concat(
|
||||
Arrays.asList(ImmutableConciseSet.union(theNullSet, bitmaps.get(0))),
|
||||
Iterables.skip(bitmaps, 1)
|
||||
),
|
||||
ConciseCompressedIndexedInts.objectStrategy
|
||||
);
|
||||
}
|
||||
}
|
||||
else {
|
||||
bumpedDictionary = false;
|
||||
}
|
||||
|
||||
final VSizeIndexed finalMultiValCol = multiValCol;
|
||||
singleValCol = VSizeIndexedInts.fromList(
|
||||
new AbstractList<Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer get(int index)
|
||||
{
|
||||
final VSizeIndexedInts ints = finalMultiValCol.get(index);
|
||||
return ints.size() == 0 ? 0 : ints.get(0) + (bumpedDictionary ? 1 : 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return finalMultiValCol.size();
|
||||
}
|
||||
},
|
||||
dictionary.size()
|
||||
);
|
||||
multiValCol = null;
|
||||
} else {
|
||||
builder.setHasMultipleValues(true);
|
||||
}
|
||||
|
||||
builder.addSerde(
|
||||
new DictionaryEncodedColumnPartSerde(dictionary, singleValCol, multiValCol, bitmaps)
|
||||
);
|
||||
|
||||
final ColumnDescriptor serdeficator = builder.build();
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
|
||||
byte[] specBytes = baos.toByteArray();
|
||||
|
||||
final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
|
||||
dimension, serdeficator.numBytes() + specBytes.length
|
||||
);
|
||||
channel.write(ByteBuffer.wrap(specBytes));
|
||||
serdeficator.write(channel);
|
||||
channel.close();
|
||||
} else if (filename.startsWith("met_")) {
|
||||
if (!filename.endsWith(String.format("%s.drd", BYTE_ORDER))) {
|
||||
skippedFiles.add(filename);
|
||||
continue;
|
||||
}
|
||||
|
||||
MetricHolder holder = MetricHolder.fromByteBuffer(v8SmooshedFiles.mapFile(filename));
|
||||
final String metric = holder.getName();
|
||||
|
||||
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
|
||||
|
||||
switch (holder.getType()) {
|
||||
case FLOAT:
|
||||
builder.setValueType(ValueType.FLOAT);
|
||||
builder.addSerde(new FloatGenericColumnPartSerde(holder.floatType, BYTE_ORDER));
|
||||
break;
|
||||
case COMPLEX:
|
||||
if (!(holder.complexType instanceof GenericIndexed)) {
|
||||
throw new ISE("Serialized complex types must be GenericIndexed objects.");
|
||||
}
|
||||
final GenericIndexed column = (GenericIndexed) holder.complexType;
|
||||
final String complexType = holder.getTypeName();
|
||||
|
||||
builder.setValueType(ValueType.COMPLEX);
|
||||
builder.addSerde(new ComplexColumnPartSerde(column, complexType));
|
||||
break;
|
||||
default:
|
||||
throw new ISE("Unknown type[%s]", holder.getType());
|
||||
}
|
||||
|
||||
final ColumnDescriptor serdeficator = builder.build();
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
|
||||
byte[] specBytes = baos.toByteArray();
|
||||
|
||||
final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
|
||||
metric, serdeficator.numBytes() + specBytes.length
|
||||
);
|
||||
channel.write(ByteBuffer.wrap(specBytes));
|
||||
serdeficator.write(channel);
|
||||
channel.close();
|
||||
} else if (String.format("time_%s.drd", BYTE_ORDER).equals(filename)) {
|
||||
CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer(
|
||||
v8SmooshedFiles.mapFile(filename), BYTE_ORDER
|
||||
);
|
||||
|
||||
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
|
||||
builder.setValueType(ValueType.LONG);
|
||||
builder.addSerde(new LongGenericColumnPartSerde(timestamps, BYTE_ORDER));
|
||||
|
||||
final ColumnDescriptor serdeficator = builder.build();
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator));
|
||||
byte[] specBytes = baos.toByteArray();
|
||||
|
||||
final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
|
||||
"__time", serdeficator.numBytes() + specBytes.length
|
||||
);
|
||||
channel.write(ByteBuffer.wrap(specBytes));
|
||||
serdeficator.write(channel);
|
||||
channel.close();
|
||||
} else if ("index.drd".equals(filename)) {
|
||||
final ByteBuffer indexBuffer = v8SmooshedFiles.mapFile(filename);
|
||||
|
||||
indexBuffer.get(); // Skip the version byte
|
||||
final GenericIndexed<String> dims = GenericIndexed.read(
|
||||
indexBuffer, GenericIndexed.stringStrategy
|
||||
);
|
||||
final GenericIndexed<String> availableMetrics = GenericIndexed.read(
|
||||
indexBuffer, GenericIndexed.stringStrategy
|
||||
);
|
||||
final Interval dataInterval = new Interval(serializerUtils.readString(indexBuffer));
|
||||
|
||||
Set<String> columns = Sets.newTreeSet();
|
||||
columns.addAll(Lists.newArrayList(dims));
|
||||
columns.addAll(Lists.newArrayList(availableMetrics));
|
||||
|
||||
GenericIndexed<String> cols = GenericIndexed.fromIterable(columns, GenericIndexed.stringStrategy);
|
||||
|
||||
final int numBytes = cols.getSerializedSize() + dims.getSerializedSize() + 16;
|
||||
final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes);
|
||||
cols.writeToChannel(writer);
|
||||
dims.writeToChannel(writer);
|
||||
serializerUtils.writeLong(writer, dataInterval.getStartMillis());
|
||||
serializerUtils.writeLong(writer, dataInterval.getEndMillis());
|
||||
writer.close();
|
||||
} else {
|
||||
skippedFiles.add(filename);
|
||||
}
|
||||
}
|
||||
log.info("Skipped files[%s]", skippedFiles);
|
||||
|
||||
v9Smoosher.close();
|
||||
}
|
||||
}
|
||||
|
||||
static interface IndexLoader
|
||||
{
|
||||
public QueryableIndex load(File inDir) throws IOException;
|
||||
}
|
||||
|
||||
static class LegacyIndexLoader implements IndexLoader
|
||||
{
|
||||
@Override
|
||||
public QueryableIndex load(File inDir) throws IOException
|
||||
{
|
||||
MMappedIndex index = IndexIO.mapDir(inDir);
|
||||
|
||||
Map<String, Column> columns = Maps.newHashMap();
|
||||
|
||||
for (String dimension : index.getAvailableDimensions()) {
|
||||
columns.put(
|
||||
dimension.toLowerCase(),
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.STRING)
|
||||
.setHasMultipleValues(true)
|
||||
.setDictionaryEncodedColumn(
|
||||
new DictionaryEncodedColumnSupplier(
|
||||
index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension))
|
||||
)
|
||||
)
|
||||
.setBitmapIndex(
|
||||
new BitmapIndexColumnPartSupplier(
|
||||
index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension)
|
||||
)
|
||||
)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
for (String metric : index.getAvailableMetrics()) {
|
||||
final MetricHolder metricHolder = index.getMetricHolder(metric);
|
||||
if (metricHolder.getType() == MetricHolder.MetricType.FLOAT) {
|
||||
columns.put(
|
||||
metric.toLowerCase(),
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.FLOAT)
|
||||
.setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType, BYTE_ORDER))
|
||||
.build()
|
||||
);
|
||||
} else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) {
|
||||
columns.put(
|
||||
metric.toLowerCase(),
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.COMPLEX)
|
||||
.setComplexColumn(
|
||||
new ComplexColumnPartSupplier(
|
||||
metricHolder.getTypeName(), (GenericIndexed) metricHolder.complexType
|
||||
)
|
||||
)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
Set<String> colSet = Sets.newTreeSet();
|
||||
for (String dimension : index.getAvailableDimensions()) {
|
||||
colSet.add(dimension.toLowerCase());
|
||||
}
|
||||
for (String metric : index.getAvailableMetrics()) {
|
||||
colSet.add(metric.toLowerCase());
|
||||
}
|
||||
|
||||
String[] cols = colSet.toArray(new String[colSet.size()]);
|
||||
|
||||
return new SimpleQueryableIndex(
|
||||
index.getDataInterval(),
|
||||
new ArrayIndexed<String>(cols, String.class),
|
||||
index.getAvailableDimensions(),
|
||||
new ColumnBuilder()
|
||||
.setType(ValueType.LONG)
|
||||
.setGenericColumn(new LongGenericColumnSupplier(index.timestamps))
|
||||
.build(),
|
||||
columns
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
static class V9IndexLoader implements IndexLoader
|
||||
{
|
||||
@Override
|
||||
public QueryableIndex load(File inDir) throws IOException
|
||||
{
|
||||
log.debug("Mapping v9 index[%s]", inDir);
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
final int theVersion = Ints.fromByteArray(Files.toByteArray(new File(inDir, "version.bin")));
|
||||
if (theVersion != V9_VERSION) {
|
||||
throw new IllegalArgumentException(String.format("Expected version[9], got[%s]", theVersion));
|
||||
}
|
||||
|
||||
SmooshedFileMapper smooshedFiles = Smoosh.map(inDir);
|
||||
|
||||
ByteBuffer indexBuffer = smooshedFiles.mapFile("index.drd");
|
||||
final GenericIndexed<String> cols = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy);
|
||||
final GenericIndexed<String> dims = GenericIndexed.read(indexBuffer, GenericIndexed.stringStrategy);
|
||||
final Interval dataInterval = new Interval(indexBuffer.getLong(), indexBuffer.getLong());
|
||||
|
||||
Map<String, Column> columns = Maps.newHashMap();
|
||||
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
for (String columnName : cols) {
|
||||
columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName)));
|
||||
}
|
||||
|
||||
final QueryableIndex index = new SimpleQueryableIndex(
|
||||
dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns
|
||||
);
|
||||
|
||||
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
|
||||
|
||||
return index;
|
||||
}
|
||||
|
||||
private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer) throws IOException
|
||||
{
|
||||
ColumnDescriptor serde = mapper.readValue(
|
||||
serializerUtils.readString(byteBuffer), ColumnDescriptor.class
|
||||
);
|
||||
return serde.read(byteBuffer);
|
||||
}
|
||||
}
|
||||
|
||||
public static File makeDimFile(File dir, String dimension)
|
||||
|
|
|
@ -39,7 +39,6 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.Channels;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
|
@ -140,7 +139,7 @@ public class MetricHolder
|
|||
break;
|
||||
case COMPLEX:
|
||||
if (strategy != null) {
|
||||
holder.complexType = GenericIndexed.readFromByteBuffer(buf, strategy);
|
||||
holder.complexType = GenericIndexed.read(buf, strategy);
|
||||
} else {
|
||||
final ComplexMetricSerde serdeForType = ComplexMetrics.getSerdeForType(holder.getTypeName());
|
||||
|
||||
|
@ -148,7 +147,7 @@ public class MetricHolder
|
|||
throw new ISE("Unknown type[%s], cannot load.", holder.getTypeName());
|
||||
}
|
||||
|
||||
holder.complexType = GenericIndexed.readFromByteBuffer(buf, serdeForType.getObjectStrategy());
|
||||
holder.complexType = GenericIndexed.read(buf, serdeForType.getObjectStrategy());
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -174,8 +173,8 @@ public class MetricHolder
|
|||
}
|
||||
}
|
||||
|
||||
private CompressedFloatsIndexedSupplier floatType = null;
|
||||
private Indexed complexType = null;
|
||||
CompressedFloatsIndexedSupplier floatType = null;
|
||||
Indexed complexType = null;
|
||||
|
||||
private MetricHolder(
|
||||
String name,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue