implement bulk get test

This commit is contained in:
xvrl 2013-01-16 19:15:43 -08:00
parent e2788187fb
commit dcaa77a883
1 changed files with 89 additions and 4 deletions

View File

@ -19,7 +19,10 @@
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 com.metamx.common.Pair;
import net.spy.memcached.CASResponse;
import net.spy.memcached.CASValue;
import net.spy.memcached.CachedData;
@ -27,6 +30,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;
@ -34,6 +38,7 @@ import org.junit.Before;
import org.junit.Test;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Iterator;
import java.util.Map;
@ -85,6 +90,28 @@ public class MemcachedCacheBrokerTest
cache.close("a");
}
@Test
public void testGetBulk() throws Exception
{
Assert.assertNull(cache.get("the", HI));
put(cache, "the", HI, 2);
put(cache, "the", HO, 10);
Pair<String, ByteBuffer> key1 = Pair.of("the", ByteBuffer.wrap(HI));
Pair<String, ByteBuffer> key2 = Pair.of("the", ByteBuffer.wrap(HO));
Map<Pair<String, ByteBuffer>, 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 void put(CacheBroker cache, String identifier, byte[] key, Integer value)
{
cache.put(identifier, key, Ints.toByteArray(value));
@ -362,9 +389,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
@ -380,9 +465,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