Merge branch 'master' of github.com:metamx/druid

This commit is contained in:
cheddar 2013-06-07 14:15:39 -07:00
commit 06f90bc2e1
10 changed files with 262 additions and 43 deletions

View File

@ -181,6 +181,10 @@
<groupId>com.metamx</groupId>
<artifactId>bytebuffer-collections</artifactId>
</dependency>
<dependency>
<groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -0,0 +1,81 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.client.cache;
import com.google.common.primitives.Ints;
import net.jpountz.lz4.LZ4Compressor;
import net.jpountz.lz4.LZ4Decompressor;
import net.jpountz.lz4.LZ4Factory;
import net.spy.memcached.transcoders.SerializingTranscoder;
import java.nio.ByteBuffer;
public class LZ4Transcoder extends SerializingTranscoder
{
private final LZ4Factory lz4Factory;
public LZ4Transcoder()
{
super();
lz4Factory = LZ4Factory.fastestJavaInstance();
}
public LZ4Transcoder(int max)
{
super(max);
lz4Factory = LZ4Factory.fastestJavaInstance();
}
@Override
protected byte[] compress(byte[] in)
{
if (in == null) {
throw new NullPointerException("Can't compress null");
}
LZ4Compressor compressor = lz4Factory.fastCompressor();
byte[] out = new byte[compressor.maxCompressedLength(in.length)];
int compressedLength = compressor.compress(in, 0, in.length, out, 0);
getLogger().debug("Compressed %d bytes to %d", in.length, compressedLength);
return ByteBuffer.allocate(Ints.BYTES + compressedLength)
.putInt(in.length)
.put(out, 0, compressedLength)
.array();
}
@Override
protected byte[] decompress(byte[] in)
{
byte[] out = null;
if(in != null) {
LZ4Decompressor decompressor = lz4Factory.decompressor();
int size = ByteBuffer.wrap(in).getInt();
out = new byte[size];
decompressor.decompress(in, Ints.BYTES, out, 0, out.length);
}
return out == null ? null : out;
}
}

View File

@ -53,9 +53,10 @@ 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);
LZ4Transcoder transcoder = new LZ4Transcoder(config.getMaxObjectSize());
// always use compression
transcoder.setCompressionThreshold(0);
return new MemcachedCache(
new MemcachedClient(

View File

@ -0,0 +1,127 @@
/*
* 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;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
*/
public class DelegatingExecutorService implements ExecutorService
{
private final ExecutorService delegate;
public DelegatingExecutorService(ExecutorService delegate)
{
this.delegate = delegate;
}
@Override
public void shutdown()
{
delegate.shutdown();
}
@Override
public List<Runnable> shutdownNow()
{
return delegate.shutdownNow();
}
@Override
public boolean isShutdown()
{
return delegate.isShutdown();
}
@Override
public boolean isTerminated()
{
return delegate.isTerminated();
}
@Override
public boolean awaitTermination(long l, TimeUnit timeUnit) throws InterruptedException
{
return delegate.awaitTermination(l, timeUnit);
}
@Override
public <T> Future<T> submit(Callable<T> tCallable)
{
return delegate.submit(tCallable);
}
@Override
public <T> Future<T> submit(Runnable runnable, T t)
{
return delegate.submit(runnable, t);
}
@Override
public Future<?> submit(Runnable runnable)
{
return delegate.submit(runnable);
}
@Override
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> callables) throws InterruptedException
{
return delegate.invokeAll(callables);
}
@Override
public <T> List<Future<T>> invokeAll(
Collection<? extends Callable<T>> callables,
long l,
TimeUnit timeUnit
) throws InterruptedException
{
return delegate.invokeAll(callables, l, timeUnit);
}
@Override
public <T> T invokeAny(Collection<? extends Callable<T>> callables) throws InterruptedException, ExecutionException
{
return delegate.invokeAny(callables);
}
@Override
public <T> T invokeAny(
Collection<? extends Callable<T>> callables,
long l,
TimeUnit timeUnit
) throws InterruptedException, ExecutionException, TimeoutException
{
return delegate.invokeAny(callables, l, timeUnit);
}
@Override
public void execute(Runnable runnable)
{
delegate.execute(runnable);
}
}

View File

@ -1,15 +1,40 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.query;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
public class MetricsEmittingExecutorService extends AbstractExecutorService
public class MetricsEmittingExecutorService extends DelegatingExecutorService
{
private final ExecutorService base;
private final ServiceEmitter emitter;
@ -21,41 +46,13 @@ public class MetricsEmittingExecutorService extends AbstractExecutorService
ServiceMetricEvent.Builder metricBuilder
)
{
super(base);
this.base = base;
this.emitter = emitter;
this.metricBuilder = metricBuilder;
}
@Override
public void shutdown()
{
base.shutdown();
}
@Override
public List<Runnable> shutdownNow()
{
return base.shutdownNow();
}
@Override
public boolean isShutdown()
{
return base.isShutdown();
}
@Override
public boolean isTerminated()
{
return base.isTerminated();
}
@Override
public boolean awaitTermination(long l, TimeUnit timeUnit) throws InterruptedException
{
return base.awaitTermination(l, timeUnit);
}
@Override
public void execute(Runnable runnable)
{

View File

@ -38,9 +38,9 @@ import java.util.concurrent.TimeUnit;
*/
public class PrioritizedExecutorService extends AbstractExecutorService
{
public static ExecutorService create(Lifecycle lifecycle, ExecutorServiceConfig config)
public static PrioritizedExecutorService create(Lifecycle lifecycle, ExecutorServiceConfig config)
{
final ExecutorService service = new PrioritizedExecutorService(
final PrioritizedExecutorService service = new PrioritizedExecutorService(
new ThreadPoolExecutor(
config.getNumThreads(),
config.getNumThreads(),

View File

@ -52,8 +52,8 @@ import java.util.concurrent.TimeoutException;
*/
public class MemcachedCacheTest
{
private static final byte[] HI = "hi".getBytes();
private static final byte[] HO = "ho".getBytes();
private static final byte[] HI = "hiiiiiiiiiiiiiiiiiii".getBytes();
private static final byte[] HO = "hooooooooooooooooooo".getBytes();
private MemcachedCache cache;
@Before
@ -124,7 +124,13 @@ public class MemcachedCacheTest
class MockMemcachedClient implements MemcachedClientIF
{
private final ConcurrentMap<String, CachedData> theMap = new ConcurrentHashMap<String, CachedData>();
private final Transcoder<Object> transcoder = new SerializingTranscoder();
private final SerializingTranscoder transcoder;
public MockMemcachedClient()
{
transcoder = new LZ4Transcoder();
transcoder.setCompressionThreshold(0);
}
@Override
public Collection<SocketAddress> getAvailableServers()

View File

@ -32,10 +32,7 @@ import com.metamx.druid.curator.announcement.Announcer;
import com.metamx.druid.indexing.common.TaskStatus;
import com.metamx.druid.indexing.common.config.IndexerZkConfig;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;
import java.util.Arrays;

View File

@ -320,6 +320,12 @@
<artifactId>jackson-mapper-asl</artifactId>
<version>1.9.11</version>
</dependency>
<dependency>
<groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId>
<version>1.1.2</version>
</dependency>
<!-- Test Scope -->
<dependency>

View File

@ -97,7 +97,7 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
final List<Monitor> monitors = getMonitors();
final QueryRunnerFactoryConglomerate conglomerate = getConglomerate();
final ExecutorService innerExecutorService = PrioritizedExecutorService.create(
final PrioritizedExecutorService innerExecutorService = PrioritizedExecutorService.create(
getLifecycle(),
getConfigFactory().buildWithReplacements(
ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing")