merge master

This commit is contained in:
Nelson Ray 2013-02-11 11:28:50 -08:00
commit e12da51057
228 changed files with 10026 additions and 3283 deletions

View File

@ -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>

View File

@ -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

View File

@ -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();

View File

@ -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);
segments.add(Pair.of(selector, descriptor));
}
}
}
descriptors.add(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 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)
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;

View File

@ -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
);
}

View File

@ -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;
}
}
}

View File

@ -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();

View File

@ -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;
}
}

View File

@ -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;
}
}
}

View File

@ -24,7 +24,7 @@ import org.skife.config.Default;
/**
*/
public abstract class MapCacheBrokerConfig
public abstract class MapCacheConfig
{
@Config("${prefix}.sizeInBytes")
@Default("0")

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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)
)
)

View File

@ -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));
}
}

View File

@ -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()
)
);

View File

@ -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

View File

@ -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);
}

View File

@ -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());
}

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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;
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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();
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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 +
'}';
}
}

View File

@ -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());
}
}

View File

@ -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 QueryRunner<Result<SegmentMetadataResultValue>> preMergeQueryDecoration(QueryRunner<Result<SegmentMetadataResultValue>> runner)
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<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());
}
};
}
}

View File

@ -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()
{

View File

@ -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);
}
}

View File

@ -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()
{

View File

@ -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();
}
}

View File

@ -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) {

View File

@ -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);

View File

@ -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);

View File

@ -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)));
}
}

View File

@ -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);
}
}

View File

@ -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

View File

@ -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>

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -37,7 +37,7 @@ import java.util.Set;
})
public interface PostAggregator
{
public boolean verifyFields(Set<String> fieldNames);
public Set<String> getDependentFields();
public Comparator getComparator();

View File

@ -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,

View File

@ -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;
}
}

View File

@ -52,6 +52,12 @@ public class Rows
{
return row.getFloatMetric(metric);
}
@Override
public String toString()
{
return row.toString();
}
};
}
}

View File

@ -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);

View File

@ -19,8 +19,6 @@
package com.metamx.druid.processing;
import java.io.Closeable;
/**
* Factory class for MetricSelectors
*/

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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
{

View File

@ -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)

View File

@ -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": "doubleSum", "fieldName": "tweets", "name": "tweets"}
],
"filter": { "type": "selector", "dimension": "lang", "value": "en" },
"intervals":["2012-10-01T00:00/2020-01-01T00"]
}

View File

@ -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>

View File

@ -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
{

View File

@ -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);

View File

@ -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)

View File

@ -31,8 +31,8 @@
"firehose": {
"type": "twitzer",
"maxEventCount": 50000,
"maxRunMinutes": 10
"maxEventCount": 500000,
"maxRunMinutes": 120
},
"plumber": {

View File

@ -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>

View File

@ -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();
}

View File

@ -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);
}
}

View File

@ -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();
}
}

View File

@ -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);
}

View File

@ -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();
}

View File

@ -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
);
}
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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());
}
}

View File

@ -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);
}

View File

@ -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
{
}
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -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());
}
}

View File

@ -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();
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -0,0 +1,11 @@
package com.metamx.druid.index.column;
/**
*/
public enum ValueType
{
FLOAT,
LONG,
STRING,
COMPLEX
}

View File

@ -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;
}
};
}
}

View File

@ -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);
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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());
}
}

View File

@ -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);
}
}

View File

@ -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());
}
}

View File

@ -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))
);
}

View File

@ -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))
);
}

View File

@ -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)

View File

@ -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)

View File

@ -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