diff --git a/.circleci/config.yml b/.circleci/config.yml
index 23e8087..5770bd6 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -6,6 +6,7 @@ jobs:
 
     docker:
       - image: circleci/openjdk:8-jdk-browsers
+      - image: circleci/redis:latest
 
     steps:
       - checkout
diff --git a/benchmark/pom.xml b/benchmark/pom.xml
index 31b0cf2..88400fc 100644
--- a/benchmark/pom.xml
+++ b/benchmark/pom.xml
@@ -65,6 +65,11 @@
       <optional>true</optional>
     </dependency>
 
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>bufferslayer-jedis</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>ch.qos.logback</groupId>
       <artifactId>logback-classic</artifactId>
diff --git a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJdbcTemplateBenchmark.java b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJdbcTemplateBenchmark.java
index f1dc6d1..443aed2 100644
--- a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJdbcTemplateBenchmark.java
+++ b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJdbcTemplateBenchmark.java
@@ -25,7 +25,7 @@ public abstract class AbstractBatchJdbcTemplateBenchmark {
   private BatchJdbcTemplate batch;
   private JdbcTemplate unbatch;
   private Reporter<Sql, Integer> reporter;
-  private static SenderProxy proxy;
+  private static SenderProxy<Sql, Integer> proxy;
   private static AtomicLong counter = new AtomicLong();
 
   private static final String CREATE_DATABASE = "CREATE DATABASE IF NOT EXISTS test";
@@ -51,7 +51,7 @@ public void setup() {
     JdbcTemplate delegate = new JdbcTemplate(dataSource);
     delegate.setDataSource(dataSource);
 
-    proxy = new SenderProxy(new JdbcTemplateSender(delegate));
+    proxy = new SenderProxy<>(new JdbcTemplateSender(delegate));
     proxy.onMessages(updated -> counter.addAndGet(updated.size()));
 
     reporter = reporter(proxy);
diff --git a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJedisBenchmark.java b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJedisBenchmark.java
new file mode 100644
index 0000000..06571d0
--- /dev/null
+++ b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractBatchJedisBenchmark.java
@@ -0,0 +1,116 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.openjdk.jmh.annotations.AuxCounters;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Group;
+import org.openjdk.jmh.annotations.GroupThreads;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+
+/**
+ * Jedis benchmark that executing jedis commands.
+ */
+public abstract class AbstractBatchJedisBenchmark {
+
+  private JedisPool unbatch;
+  private BatchJedis batch;
+  private Reporter<RedisCommand, Object> reporter;
+  private static SenderProxy<RedisCommand, Object> proxy;
+  private static AtomicLong counter = new AtomicLong();
+
+  static String propertyOr(String key, String fallback) {
+    return System.getProperty(key, fallback);
+  }
+
+  protected abstract Reporter<RedisCommand, Object> reporter(Sender<RedisCommand, Object> sender);
+
+  @Setup
+  public void setup() {
+    unbatch = new JedisPool(propertyOr("redisHost", "127.0.0.1"),
+        Integer.parseInt(propertyOr("redisPort", "6379")));
+
+    proxy = new SenderProxy<>(new JedisSender(unbatch));
+    proxy.onMessages(updated -> counter.addAndGet(updated.size()));
+
+    reporter = reporter(proxy);
+    batch = new BatchJedis(unbatch, reporter);
+  }
+
+  @TearDown(Level.Iteration)
+  public void flushDB() {
+    try (Jedis jedis = unbatch.getResource()) {
+      jedis.flushDB();
+    }
+  }
+
+  @AuxCounters
+  @State(Scope.Thread)
+  public static class AtomicLongCounter {
+
+    public long updated() {
+      return counter.get();
+    }
+
+    @Setup(Level.Iteration)
+    public void clean() {
+      counter.set(0);
+    }
+  }
+
+  @State(Scope.Benchmark)
+  public static class Lagging {
+
+    @Setup(Level.Iteration)
+    public void lag() throws InterruptedException {
+      TimeUnit.SECONDS.sleep(1);
+    }
+  }
+
+  static String randomString() {
+    return String.valueOf(ThreadLocalRandom.current().nextLong());
+  }
+
+  @Benchmark @Group("no_contention_batched") @GroupThreads(1)
+  public void no_contention_batched_set(Lagging l, AtomicLongCounter counters) {
+    batch.set(randomString(), randomString());
+  }
+
+  @Benchmark @Group("no_contention_unbatched") @GroupThreads(1)
+  public void no_contention_unbatched_set(Lagging l) {
+    try (Jedis jedis = unbatch.getResource()) {
+      jedis.set(randomString(), randomString());
+    }
+  }
+
+  @Benchmark @Group("mild_contention_batched") @GroupThreads(2)
+  public void mild_contention_batched_set(Lagging l, AtomicLongCounter counters) {
+    batch.set(randomString(), randomString());
+  }
+
+  @Benchmark @Group("mild_contention_unbatched") @GroupThreads(2)
+  public void mild_contention_unbatched_set(Lagging l) {
+    try (Jedis jedis = unbatch.getResource()) {
+      jedis.set(randomString(), randomString());
+    }
+  }
+
+  @Benchmark @Group("high_contention_batched") @GroupThreads(8)
+  public void high_contention_batched_set(Lagging l, AtomicLongCounter counters) {
+    batch.set(randomString(), randomString());
+  }
+
+  @Benchmark @Group("high_contention_unbatched") @GroupThreads(8)
+  public void high_contention_unbatched_set(Lagging l) {
+    try (Jedis jedis = unbatch.getResource()) {
+      jedis.set(randomString(), randomString());
+    }
+  }
+}
diff --git a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractTimeUsedComparison.java b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractTimeUsedComparison.java
index 7158e5d..f414328 100644
--- a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractTimeUsedComparison.java
+++ b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AbstractTimeUsedComparison.java
@@ -21,7 +21,7 @@ static String propertyOr(String key, String fallback) {
   protected void run() throws Exception {
     BatchJdbcTemplate batch;
     JdbcTemplate unbatch;
-    SenderProxy proxy;
+    SenderProxy<Sql, Integer> proxy;
 
     DriverManagerDataSource dataSource = new DriverManagerDataSource();
     dataSource.setDriverClassName("com.mysql.jdbc.Driver");
@@ -42,7 +42,7 @@ protected void run() throws Exception {
 
     CountDownLatch countDown = new CountDownLatch(1);
 
-    proxy = new SenderProxy(new JdbcTemplateSender(delegate));
+    proxy = new SenderProxy<>(new JdbcTemplateSender(delegate));
     proxy.onMessages(updated -> {
       if (counter.addAndGet(updated.size()) == 5050) {
         countDown.countDown();
diff --git a/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AsyncBatchJedisBenchmark.java b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AsyncBatchJedisBenchmark.java
new file mode 100644
index 0000000..5ad8fc8
--- /dev/null
+++ b/benchmark/src/main/java/io/github/tramchamploo/bufferslayer/AsyncBatchJedisBenchmark.java
@@ -0,0 +1,38 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.concurrent.TimeUnit;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+@Measurement(iterations = 5, time = 1)
+@Warmup(iterations = 3, time = 1)
+@Fork(3)
+@BenchmarkMode(Mode.Throughput)
+@OutputTimeUnit(TimeUnit.SECONDS)
+@State(Scope.Group)
+public class AsyncBatchJedisBenchmark extends AbstractBatchJedisBenchmark {
+
+  protected Reporter<RedisCommand, Object> reporter(Sender<RedisCommand, Object> sender) {
+    return AsyncReporter.builder(sender)
+        .pendingKeepalive(1, TimeUnit.SECONDS)
+        .build();
+  }
+
+  public static void main(String[] args) throws RunnerException {
+    Options opt = new OptionsBuilder()
+        .include(".*" + AsyncBatchJedisBenchmark.class.getSimpleName() + ".*")
+        .build();
+
+    new Runner(opt).run();
+  }
+}
diff --git a/boundedqueue/src/test/java/io/github/tramchamploo/bufferslayer/AsyncReporterTest.java b/boundedqueue/src/test/java/io/github/tramchamploo/bufferslayer/AsyncReporterTest.java
index 9753f3b..29f6240 100644
--- a/boundedqueue/src/test/java/io/github/tramchamploo/bufferslayer/AsyncReporterTest.java
+++ b/boundedqueue/src/test/java/io/github/tramchamploo/bufferslayer/AsyncReporterTest.java
@@ -67,6 +67,8 @@ public void flushIfExceedMaxSize() throws InterruptedException {
     assertTrue(countDown.await(300, TimeUnit.MILLISECONDS));
     assertEquals(50, sender.sent.size());
 
+    // wait for the queue to be released
+    Thread.sleep(100);
     // make sure the queue is released
     assertEquals(0, reporter.synchronizer.deque.size());
     assertEquals(0, reporter.synchronizer.keyToReady.size());
diff --git a/circle.yml b/circle.yml
new file mode 100644
index 0000000..662724d
--- /dev/null
+++ b/circle.yml
@@ -0,0 +1,19 @@
+machine:
+  java:
+    version: oraclejdk8
+  services:
+    - redis
+
+dependencies:
+  pre:
+    - openssl aes-256-cbc -d -in .buildscript/secret-env-cipher -k $KEY >> ~/.circlerc
+  override:
+    - mvn --fail-never dependency:go-offline || true
+  cache_directories:
+    - "~/.m2"
+
+test:
+  post:
+    - mkdir -p $CIRCLE_TEST_REPORTS/junit/
+    - find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} $CIRCLE_TEST_REPORTS/junit/ \;
+    - .buildscript/release.sh
\ No newline at end of file
diff --git a/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/CompositeFuture.java b/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/CompositeFuture.java
index 7940f63..4bafb5b 100644
--- a/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/CompositeFuture.java
+++ b/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/CompositeFuture.java
@@ -14,7 +14,7 @@ public abstract class CompositeFuture extends AbstractFuture<CompositeFuture> {
    * When the list is empty, the returned future will be already completed.
    */
   public static CompositeFuture all(List<? extends Future<?>> futures) {
-    return DefaultCompositeFuture.all(futures.toArray(new Future[futures.size()]));
+    return DefaultCompositeFuture.all(futures.toArray(new Future[0]));
   }
 
   /**
diff --git a/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/Promises.java b/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/Promises.java
index ca9004b..92058d3 100644
--- a/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/Promises.java
+++ b/core/src/main/java/io/github/tramchamploo/bufferslayer/internal/Promises.java
@@ -6,6 +6,7 @@
 import io.github.tramchamploo.bufferslayer.Message;
 import io.github.tramchamploo.bufferslayer.MessageDroppedException;
 import io.github.tramchamploo.bufferslayer.OverflowStrategy.Strategy;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -19,7 +20,12 @@ public static <R> void allSuccess(List<R> result, List<MessagePromise<R>> promis
     for (int i = 0; i < result.size(); i++) {
       MessagePromise<R> promise = promises.get(i);
       R ret = result.get(i);
-      promise.setSuccess(ret);
+      if (ret instanceof Throwable) {
+        promise.setFailure(MessageDroppedException.dropped((Throwable) ret,
+            Collections.singletonList(promise.message())));
+      } else {
+        promise.setSuccess(ret);
+      }
     }
   }
 
diff --git a/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java b/core/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java
similarity index 53%
rename from jdbc/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java
rename to core/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java
index 3d38d58..ac55876 100644
--- a/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java
+++ b/core/src/test/java/io/github/tramchamploo/bufferslayer/SenderProxy.java
@@ -7,13 +7,13 @@
 /**
  * Delegate sending and trigger onMessages afterwards
  */
-public class SenderProxy implements Sender<Sql, Integer> {
+public class SenderProxy<M extends Message, R> implements Sender<M, R> {
 
-  private AtomicBoolean closed = new AtomicBoolean(false);
-  private Consumer<List<Integer>> onMessages = messages -> { };
-  final Sender<Sql, Integer> delegate;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+  private Consumer<List<M>> onMessages = messages -> { };
+  private final Sender<M, R> delegate;
 
-  public SenderProxy(Sender<Sql, Integer> delegate) {
+  SenderProxy(Sender<M, R> delegate) {
     this.delegate = delegate;
   }
 
@@ -28,16 +28,16 @@ public void close() {
   }
 
   @Override
-  public List<Integer> send(List<Sql> messages) {
+  public List<R> send(List<M> messages) {
     if (closed.get()) {
       throw new IllegalStateException("Closed!");
     }
-    List<Integer> sent = delegate.send(messages);
-    onMessages.accept(sent);
+    List<R> sent = delegate.send(messages);
+    onMessages.accept(messages);
     return sent;
   }
 
-  public void onMessages(Consumer<List<Integer>> onMessages) {
+  public void onMessages(Consumer<List<M>> onMessages) {
     this.onMessages = onMessages;
   }
 }
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 8810a08..09ea68c 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -15,6 +15,13 @@
       <artifactId>bufferslayer-core</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>bufferslayer-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
     <dependency>
       <groupId>${project.groupId}</groupId>
       <artifactId>bufferslayer-boundedqueue</artifactId>
diff --git a/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/BatchJdbcTemplateTest.java b/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/BatchJdbcTemplateTest.java
index 3bd72cc..7d54436 100644
--- a/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/BatchJdbcTemplateTest.java
+++ b/jdbc/src/test/java/io/github/tramchamploo/bufferslayer/BatchJdbcTemplateTest.java
@@ -129,7 +129,7 @@ public void singleKey() {
 
   @Test
   public void updateFailed() throws InterruptedException {
-    SenderProxy sender = new SenderProxy(new JdbcTemplateSender(underlying));
+    SenderProxy<Sql, Integer> sender = new SenderProxy<>(new JdbcTemplateSender(underlying));
     RuntimeException ex = new RuntimeException();
     sender.onMessages(messages -> { throw ex; });
     reporter = AsyncReporter.builder(sender).messageTimeout(10, TimeUnit.MILLISECONDS).build();
diff --git a/jedis/pom.xml b/jedis/pom.xml
new file mode 100644
index 0000000..a18deb7
--- /dev/null
+++ b/jedis/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>bufferslayer-parent</artifactId>
+    <groupId>io.github.tramchamploo</groupId>
+    <version>1.4.4-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>bufferslayer-jedis</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>bufferslayer-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>bufferslayer-boundedqueue</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>redis.clients</groupId>
+      <artifactId>jedis</artifactId>
+      <version>2.9.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BatchJedis.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BatchJedis.java
new file mode 100644
index 0000000..6973e84
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BatchJedis.java
@@ -0,0 +1,2157 @@
+package io.github.tramchamploo.bufferslayer;
+
+import static io.github.tramchamploo.bufferslayer.ResponseUtil.transformResponse;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.Blpop;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.Brpop;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.Del;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.Exists;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.MGet;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.MSet;
+import io.github.tramchamploo.bufferslayer.MultiKeyRedisCommand.MSetNX;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Append;
+import io.github.tramchamploo.bufferslayer.RedisCommand.BitPos;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Decr;
+import io.github.tramchamploo.bufferslayer.RedisCommand.DecrBy;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Echo;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Expire;
+import io.github.tramchamploo.bufferslayer.RedisCommand.ExpireAt;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Get;
+import io.github.tramchamploo.bufferslayer.RedisCommand.GetBit;
+import io.github.tramchamploo.bufferslayer.RedisCommand.GetRange;
+import io.github.tramchamploo.bufferslayer.RedisCommand.GetSet;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HDel;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HExists;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HGet;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HGetAll;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HIncrBy;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HKeys;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HLen;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HMGet;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HMSet;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HSet;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HSetNX;
+import io.github.tramchamploo.bufferslayer.RedisCommand.HVals;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Incr;
+import io.github.tramchamploo.bufferslayer.RedisCommand.IncrBy;
+import io.github.tramchamploo.bufferslayer.RedisCommand.LIndex;
+import io.github.tramchamploo.bufferslayer.RedisCommand.LInsert;
+import io.github.tramchamploo.bufferslayer.RedisCommand.LLen;
+import io.github.tramchamploo.bufferslayer.RedisCommand.LPop;
+import io.github.tramchamploo.bufferslayer.RedisCommand.RPopLpush;
+import io.github.tramchamploo.bufferslayer.RedisCommand.Rename;
+import io.github.tramchamploo.bufferslayer.RedisCommand.RenameNX;
+import io.github.tramchamploo.bufferslayer.internal.MessageFuture;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import redis.clients.jedis.BinaryClient.LIST_POSITION;
+import redis.clients.jedis.BinaryJedisPubSub;
+import redis.clients.jedis.BitOP;
+import redis.clients.jedis.Client;
+import redis.clients.jedis.DebugParams;
+import redis.clients.jedis.GeoCoordinate;
+import redis.clients.jedis.GeoRadiusResponse;
+import redis.clients.jedis.GeoUnit;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster.Reset;
+import redis.clients.jedis.JedisMonitor;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.JedisPubSub;
+import redis.clients.jedis.Pipeline;
+import redis.clients.jedis.PipelineBlock;
+import redis.clients.jedis.ScanParams;
+import redis.clients.jedis.ScanResult;
+import redis.clients.jedis.SortingParams;
+import redis.clients.jedis.Transaction;
+import redis.clients.jedis.TransactionBlock;
+import redis.clients.jedis.Tuple;
+import redis.clients.jedis.ZParams;
+import redis.clients.jedis.params.geo.GeoRadiusParam;
+import redis.clients.jedis.params.sortedset.ZAddParams;
+import redis.clients.jedis.params.sortedset.ZIncrByParams;
+import redis.clients.util.Pool;
+import redis.clients.util.SafeEncoder;
+import redis.clients.util.Slowlog;
+
+/**
+ * {@link Jedis} that execute commands in pipeline
+ */
+@SuppressWarnings("unchecked")
+public class BatchJedis {
+
+  private final JedisPool jedisPool;
+  private final Reporter<RedisCommand, ?> reporter;
+
+  @VisibleForTesting
+  BatchJedis(JedisPool jedisPool, Reporter<RedisCommand, ?> reporter) {
+    this.jedisPool = jedisPool;
+    this.reporter = reporter;
+  }
+
+  public BatchJedis(JedisPool jedisPool) {
+    this.jedisPool = jedisPool;
+    this.reporter = AsyncReporter.builder(new JedisSender(jedisPool)).build();
+  }
+
+  public MessageFuture<String> set(String key, String value) {
+    return (MessageFuture<String>) reporter.report(
+        new RedisCommand.Set(SafeEncoder.encode(key), SafeEncoder.encode(value), null, null, 0L));
+  }
+
+  public MessageFuture<String> set(String key, String value, String nxxx, String expx, long time) {
+    return (MessageFuture<String>) reporter.report(
+        new RedisCommand.Set(SafeEncoder.encode(key), SafeEncoder.encode(value), SafeEncoder.encode(nxxx), SafeEncoder.encode(expx), time));
+  }
+
+  public MessageFuture<String> get(String key) {
+    return transformResponse(
+        reporter.report(new Get(SafeEncoder.encode(key))), BuilderFactory.STRING);
+  }
+
+  public MessageFuture<Long> exists(String... keys) {
+    return (MessageFuture<Long>) reporter.report(new Exists(SafeEncoder.encodeMany(keys)));
+  }
+
+  public MessageFuture<Boolean> exists(String key) {
+    return transformResponse(
+        reporter.report(new Exists(SafeEncoder.encode(key))), BuilderFactory.BOOLEAN);
+  }
+
+  public MessageFuture<Long> del(String... keys) {
+    return (MessageFuture<Long>) reporter.report(new Del(SafeEncoder.encodeMany(keys)));
+  }
+
+  public MessageFuture<Long> del(String key) {
+    return (MessageFuture<Long>) reporter.report(new Del(SafeEncoder.encode(key)));
+  }
+
+  public String type(String key) {
+    return null; // TODO
+  }
+
+  public Set<String> keys(String pattern) {
+    return null; // TODO
+  }
+
+  public String randomKey() {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> rename(String oldkey, String newkey) {
+    return (MessageFuture<String>) reporter.report(
+        new Rename(SafeEncoder.encode(oldkey), SafeEncoder.encode(newkey)));
+  }
+
+  public MessageFuture<Long> renamenx(String oldkey, String newkey) {
+    return (MessageFuture<Long>) reporter.report(
+        new RenameNX(SafeEncoder.encode(oldkey), SafeEncoder.encode(newkey)));
+  }
+
+  public MessageFuture<Long> expire(String key, int seconds) {
+    return (MessageFuture<Long>) reporter.report(new Expire(SafeEncoder.encode(key), seconds));
+  }
+
+  public MessageFuture<Long> expireAt(String key, long unixTime) {
+    return (MessageFuture<Long>) reporter.report(new ExpireAt(SafeEncoder.encode(key), unixTime));
+  }
+
+  public Long ttl(String key) {
+    return null; // TODO
+  }
+
+  public Long move(String key, int dbIndex) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> getSet(String key, String value) {
+    return transformResponse(reporter.report(
+        new GetSet(SafeEncoder.encode(key), SafeEncoder.encode(value))), BuilderFactory.STRING);
+  }
+
+  public MessageFuture<List<String>> mget(String... keys) {
+    return transformResponse(reporter.report(
+        new MGet(SafeEncoder.encodeMany(keys))), BuilderFactory.STRING_LIST);
+  }
+
+  public Long setnx(String key, String value) {
+    return null; // TODO
+  }
+
+  public String setex(String key, int seconds, String value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> mset(String... keysvalues) {
+    return (MessageFuture<String>) reporter.report(new MSet(SafeEncoder.encodeMany(keysvalues)));
+  }
+
+  public MessageFuture<Long> msetnx(String... keysvalues) {
+    return (MessageFuture<Long>) reporter.report(new MSetNX(SafeEncoder.encodeMany(keysvalues)));
+  }
+
+  public MessageFuture<Long> decrBy(String key, long integer) {
+    return (MessageFuture<Long>) reporter.report(new DecrBy(SafeEncoder.encode(key), integer));
+  }
+
+  public MessageFuture<Long> decr(String key) {
+    return (MessageFuture<Long>) reporter.report(new Decr(SafeEncoder.encode(key)));
+  }
+
+  public MessageFuture<Long> incrBy(String key, long integer) {
+    return (MessageFuture<Long>) reporter.report(
+        new IncrBy(SafeEncoder.encode(key), integer));
+  }
+
+  public Double incrByFloat(String key, double value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> incr(String key) {
+    return (MessageFuture<Long>) reporter.report(new Incr(SafeEncoder.encode(key)));
+  }
+
+  public MessageFuture<Long> append(String key, String value) {
+    return (MessageFuture<Long>) reporter.report(
+        new Append(SafeEncoder.encode(key), SafeEncoder.encode(value)));
+  }
+
+  public String substr(String key, int start, int end) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> hset(String key, String field, String value) {
+    return hset(SafeEncoder.encode(key), SafeEncoder.encode(field), SafeEncoder.encode(value));
+  }
+
+  public MessageFuture<String> hget(String key, String field) {
+    return transformResponse(reporter.report(
+        new HGet(SafeEncoder.encode(key), SafeEncoder.encode(field))), BuilderFactory.STRING);
+  }
+
+  public MessageFuture<Long> hsetnx(String key, String field, String value) {
+    return hsetnx(SafeEncoder.encode(key), SafeEncoder.encode(field), SafeEncoder.encode(value));
+  }
+
+  public MessageFuture<String> hmset(String key, Map<String, String> hash) {
+    final Map<byte[], byte[]> bhash = new HashMap<>(hash.size());
+    for (final Entry<String, String> entry : hash.entrySet()) {
+      bhash.put(SafeEncoder.encode(entry.getKey()), SafeEncoder.encode(entry.getValue()));
+    }
+    return hmset(SafeEncoder.encode(key), bhash);
+  }
+
+  public MessageFuture<List<String>> hmget(String key, String... fields) {
+    return transformResponse(reporter.report(
+        new HMGet(SafeEncoder.encode(key), SafeEncoder.encodeMany(fields))), BuilderFactory.STRING_LIST);
+  }
+
+  public MessageFuture<Long> hincrBy(String key, String field, long value) {
+    return (MessageFuture<Long>) reporter.report(
+        new HIncrBy(SafeEncoder.encode(key), SafeEncoder.encode(field), value));
+  }
+
+  public Double hincrByFloat(String key, String field, double value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Boolean> hexists(String key, String field) {
+    return (MessageFuture<Boolean>) reporter.report(
+        new HExists(SafeEncoder.encode(key), SafeEncoder.encode(field)));
+  }
+
+  public MessageFuture<Long> hdel(String key, String... fields) {
+    return (MessageFuture<Long>) reporter.report(
+        new HDel(SafeEncoder.encode(key), SafeEncoder.encodeMany(fields)));
+  }
+
+  public MessageFuture<Long> hlen(String key) {
+    return (MessageFuture<Long>) reporter.report(new HLen(SafeEncoder.encode(key)));
+  }
+
+  public MessageFuture<Set<String>> hkeys(String key) {
+    return transformResponse(
+        reporter.report(new HKeys(SafeEncoder.encode(key))), BuilderFactory.STRING_SET);
+  }
+
+  public MessageFuture<List<String>> hvals(String key) {
+    return transformResponse(reporter.report(new HVals(SafeEncoder.encode(key))),
+        BuilderFactory.STRING_LIST);
+  }
+
+  public MessageFuture<Map<String, String>> hgetAll(String key) {
+    return transformResponse(
+        reporter.report(new HGetAll(SafeEncoder.encode(key))), BuilderFactory.STRING_MAP);
+  }
+
+  public Long rpush(String key, String... strings) {
+    return null; // TODO
+  }
+
+  public Long lpush(String key, String... strings) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> llen(String key) {
+    return (MessageFuture<Long>) reporter.report(new LLen(SafeEncoder.encode(key)));
+  }
+
+  public List<String> lrange(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public String ltrim(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> lindex(String key, long index) {
+    return transformResponse(
+        reporter.report(new LIndex(SafeEncoder.encode(key), index)), BuilderFactory.STRING);
+  }
+
+  public String lset(String key, long index, String value) {
+    return null; // TODO
+  }
+
+  public Long lrem(String key, long count, String value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> lpop(String key) {
+    return transformResponse(reporter.report(new LPop(SafeEncoder.encode(key))), BuilderFactory.STRING);
+  }
+
+  public String rpop(String key) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> rpoplpush(String srckey, String dstkey) {
+    return transformResponse(reporter.report(
+        new RPopLpush(SafeEncoder.encode(srckey), SafeEncoder.encode(dstkey))), BuilderFactory.STRING);
+  }
+
+  public Long sadd(String key, String... members) {
+    return null; // TODO
+  }
+
+  public Set<String> smembers(String key) {
+    return null; // TODO
+  }
+
+  public Long srem(String key, String... members) {
+    return null; // TODO
+  }
+
+  public String spop(String key) {
+    return null; // TODO
+  }
+
+  public Set<String> spop(String key, long count) {
+    return null; // TODO
+  }
+
+  public Long smove(String srckey, String dstkey, String member) {
+    return null; // TODO
+  }
+
+  public Long scard(String key) {
+    return null; // TODO
+  }
+
+  public Boolean sismember(String key, String member) {
+    return null; // TODO
+  }
+
+  public Set<String> sinter(String... keys) {
+    return null; // TODO
+  }
+
+  public Long sinterstore(String dstkey, String... keys) {
+    return null; // TODO
+  }
+
+  public Set<String> sunion(String... keys) {
+    return null; // TODO
+  }
+
+  public Long sunionstore(String dstkey, String... keys) {
+    return null; // TODO
+  }
+
+  public Set<String> sdiff(String... keys) {
+    return null; // TODO
+  }
+
+  public Long sdiffstore(String dstkey, String... keys) {
+    return null; // TODO
+  }
+
+  public String srandmember(String key) {
+    return null; // TODO
+  }
+
+  public List<String> srandmember(String key, int count) {
+    return null; // TODO
+  }
+
+  public Long zadd(String key, double score, String member) {
+    return null; // TODO
+  }
+
+  public Long zadd(String key, double score, String member,
+      ZAddParams params) {
+    return null; // TODO
+  }
+
+  public Long zadd(String key, Map<String, Double> scoreMembers) {
+    return null; // TODO
+  }
+
+  public Long zadd(String key, Map<String, Double> scoreMembers,
+      ZAddParams params) {
+    return null; // TODO
+  }
+
+  public Set<String> zrange(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long zrem(String key, String... members) {
+    return null; // TODO
+  }
+
+  public Double zincrby(String key, double score, String member) {
+    return null; // TODO
+  }
+
+  public Double zincrby(String key, double score, String member,
+      ZIncrByParams params) {
+    return null; // TODO
+  }
+
+  public Long zrank(String key, String member) {
+    return null; // TODO
+  }
+
+  public Long zrevrank(String key, String member) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrange(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeWithScores(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeWithScores(String key, long start,
+      long end) {
+    return null; // TODO
+  }
+
+  public Long zcard(String key) {
+    return null; // TODO
+  }
+
+  public Double zscore(String key, String member) {
+    return null; // TODO
+  }
+
+  public String watch(String... keys) {
+    return null; // TODO
+  }
+
+  public List<String> sort(String key) {
+    return null; // TODO
+  }
+
+  public List<String> sort(String key, SortingParams sortingParameters) {
+    return null; // TODO
+  }
+
+  public MessageFuture<List<String>> blpop(int timeout, String... keys) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Blpop(timeout, SafeEncoder.encodeMany(keys)));
+  }
+
+  public MessageFuture<List<String>> blpop(String... args) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Blpop(0, SafeEncoder.encodeMany(args)));
+  }
+
+  public MessageFuture<List<String>> brpop(String... args) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Brpop(0, SafeEncoder.encodeMany(args)));
+  }
+
+  @Deprecated
+  public MessageFuture<List<String>> blpop(String arg) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Blpop(0, SafeEncoder.encode(arg)));
+  }
+
+  @Deprecated
+  public MessageFuture<List<String>> brpop(String arg) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Brpop(0, SafeEncoder.encode(arg)));
+  }
+
+  public Long sort(String key, SortingParams sortingParameters,
+      String dstkey) {
+    return null; // TODO
+  }
+
+  public Long sort(String key, String dstkey) {
+    return null; // TODO
+  }
+
+  public MessageFuture<List<String>> brpop(int timeout, String... keys) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Brpop(timeout, SafeEncoder.encodeMany(keys)));
+  }
+
+  public Long zcount(String key, double min, double max) {
+    return null; // TODO
+  }
+
+  public Long zcount(String key, String min, String max) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByScore(String key, double min, double max) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByScore(String key, String min, String max) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByScore(String key, double min, double max, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByScore(String key, String min, String max, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(String key, double min,
+      double max) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(String key,
+      String min, String max) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(String key, double min,
+      double max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(String key,
+      String min, String max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByScore(String key, double max, double min) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByScore(String key, String max, String min) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByScore(String key, double max, double min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(String key, double max,
+      double min) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(String key, double max,
+      double min, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(String key,
+      String max, String min, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByScore(String key, String max, String min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(String key,
+      String max, String min) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByRank(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByScore(String key, double start, double end) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByScore(String key, String start, String end) {
+    return null; // TODO
+  }
+
+  public Long zunionstore(String dstkey, String... sets) {
+    return null; // TODO
+  }
+
+  public Long zunionstore(String dstkey, ZParams params, String... sets) {
+    return null; // TODO
+  }
+
+  public Long zinterstore(String dstkey, String... sets) {
+    return null; // TODO
+  }
+
+  public Long zinterstore(String dstkey, ZParams params, String... sets) {
+    return null; // TODO
+  }
+
+  public Long zlexcount(String key, String min, String max) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByLex(String key, String min, String max) {
+    return null; // TODO
+  }
+
+  public Set<String> zrangeByLex(String key, String min, String max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByLex(String key, String max, String min) {
+    return null; // TODO
+  }
+
+  public Set<String> zrevrangeByLex(String key, String max, String min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByLex(String key, String min, String max) {
+    return null; // TODO
+  }
+
+  public Long strlen(String key) {
+    return null; // TODO
+  }
+
+  public Long lpushx(String key, String... string) {
+    return null; // TODO
+  }
+
+  public Long persist(String key) {
+    return null; // TODO
+  }
+
+  public Long rpushx(String key, String... string) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> echo(String string) {
+    return transformResponse(
+        reporter.report(new Echo(SafeEncoder.encode(string))), BuilderFactory.STRING);
+  }
+
+  public MessageFuture<Long> linsert(String key, LIST_POSITION where,
+      String pivot, String value) {
+    return (MessageFuture<Long>) reporter.report(
+        new LInsert(SafeEncoder.encode(key), where, SafeEncoder.encode(pivot), SafeEncoder.encode(value)));
+  }
+
+  public String brpoplpush(String source, String destination, int timeout) {
+    return null; // TODO
+  }
+
+  public Boolean setbit(String key, long offset, boolean value) {
+    return null; // TODO
+  }
+
+  public Boolean setbit(String key, long offset, String value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Boolean> getbit(String key, long offset) {
+    return (MessageFuture<Boolean>) reporter.report(new GetBit(SafeEncoder.encode(key), offset));
+  }
+
+  public Long setrange(String key, long offset, String value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> getrange(String key, long startOffset, long endOffset) {
+    return (MessageFuture<String>) reporter.report(new GetRange(key, startOffset, endOffset));
+  }
+
+  public MessageFuture<Long> bitpos(String key, boolean value) {
+    return (MessageFuture<Long>) reporter.report(
+        new BitPos(SafeEncoder.encode(key), value, new BitPosParams()));
+  }
+
+  public MessageFuture<Long> bitpos(String key, boolean value, redis.clients.jedis.BitPosParams params) {
+    return (MessageFuture<Long>) reporter.report(
+        new BitPos(SafeEncoder.encode(key), value, params));
+  }
+
+  public List<String> configGet(String pattern) {
+    return null; // TODO
+  }
+
+  public String configSet(String parameter, String value) {
+    return null; // TODO
+  }
+
+  public Object eval(String script, int keyCount, String... params) {
+    return null; // TODO
+  }
+
+  public void subscribe(JedisPubSub jedisPubSub, String... channels) {
+    // TODO
+  }
+
+  public Long publish(String channel, String message) {
+    return null; // TODO
+  }
+
+  public void psubscribe(JedisPubSub jedisPubSub, String... patterns) {
+    // TODO
+  }
+
+  public Object eval(String script, List<String> keys,
+      List<String> args) {
+    return null; // TODO
+  }
+
+  public Object eval(String script) {
+    return null; // TODO
+  }
+
+  public Object evalsha(String script) {
+    return null; // TODO
+  }
+
+  public Object evalsha(String sha1, List<String> keys,
+      List<String> args) {
+    return null; // TODO
+  }
+
+  public Object evalsha(String sha1, int keyCount, String... params) {
+    return null; // TODO
+  }
+
+  public Boolean scriptExists(String sha1) {
+    return null; // TODO
+  }
+
+  public List<Boolean> scriptExists(String... sha1) {
+    return null; // TODO
+  }
+
+  public String scriptLoad(String script) {
+    return null; // TODO
+  }
+
+  public List<Slowlog> slowlogGet() {
+    return null; // TODO
+  }
+
+  public List<Slowlog> slowlogGet(long entries) {
+    return null; // TODO
+  }
+
+  public Long objectRefcount(String string) {
+    return null; // TODO
+  }
+
+  public String objectEncoding(String string) {
+    return null; // TODO
+  }
+
+  public Long objectIdletime(String string) {
+    return null; // TODO
+  }
+
+  public Long bitcount(String key) {
+    return null; // TODO
+  }
+
+  public Long bitcount(String key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long bitop(BitOP op, String destKey, String... srcKeys) {
+    return null; // TODO
+  }
+
+  public List<Map<String, String>> sentinelMasters() {
+    return null; // TODO
+  }
+
+  public List<String> sentinelGetMasterAddrByName(String masterName) {
+    return null; // TODO
+  }
+
+  public Long sentinelReset(String pattern) {
+    return null; // TODO
+  }
+
+  public List<Map<String, String>> sentinelSlaves(String masterName) {
+    return null; // TODO
+  }
+
+  public String sentinelFailover(String masterName) {
+    return null; // TODO
+  }
+
+  public String sentinelMonitor(String masterName, String ip, int port, int quorum) {
+    return null; // TODO
+  }
+
+  public String sentinelRemove(String masterName) {
+    return null; // TODO
+  }
+
+  public String sentinelSet(String masterName,
+      Map<String, String> parameterMap) {
+    return null; // TODO
+  }
+
+  public byte[] dump(String key) {
+    return null; // TODO
+  }
+
+  public String restore(String key, int ttl, byte[] serializedValue) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public Long pexpire(String key, int milliseconds) {
+    return null; // TODO
+  }
+
+  public Long pexpire(String key, long milliseconds) {
+    return null; // TODO
+  }
+
+  public Long pexpireAt(String key, long millisecondsTimestamp) {
+    return null; // TODO
+  }
+
+  public Long pttl(String key) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public String psetex(String key, int milliseconds, String value) {
+    return null; // TODO
+  }
+
+  public String psetex(String key, long milliseconds, String value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> set(String key, String value, String nxxx) {
+    return (MessageFuture<String>) reporter.report(
+        new RedisCommand.Set(SafeEncoder.encode(key), SafeEncoder.encode(value), SafeEncoder.encode(nxxx), null, 0L));
+  }
+
+  public MessageFuture<String> set(String key, String value, String nxxx, String expx, int time) {
+    return (MessageFuture<String>) reporter.report(
+        new RedisCommand.Set(SafeEncoder.encode(key), SafeEncoder.encode(value), SafeEncoder.encode(nxxx), SafeEncoder.encode(expx), time));
+  }
+
+  public String clientKill(String client) {
+    return null; // TODO
+  }
+
+  public String clientSetname(String name) {
+    return null; // TODO
+  }
+
+  public String migrate(String host, int port, String key, int destinationDb, int timeout) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<String> scan(int cursor) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<String> scan(int cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<Entry<String, String>> hscan(
+      String key, int cursor) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<Entry<String, String>> hscan(
+      String key, int cursor, ScanParams params) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<String> sscan(String key, int cursor) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<String> sscan(String key, int cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<Tuple> zscan(String key, int cursor) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public ScanResult<Tuple> zscan(String key, int cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<String> scan(String cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<String> scan(String cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<Entry<String, String>> hscan(
+      String key, String cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<Entry<String, String>> hscan(
+      String key, String cursor, ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<String> sscan(String key, String cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<String> sscan(String key, String cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<Tuple> zscan(String key,
+      String cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<Tuple> zscan(String key,
+      String cursor, ScanParams params) {
+    return null; // TODO
+  }
+
+  public String clusterNodes() {
+    return null; // TODO
+  }
+
+  public String readonly() {
+    return null; // TODO
+  }
+
+  public String clusterMeet(String ip, int port) {
+    return null; // TODO
+  }
+
+  public String clusterReset(Reset resetType) {
+    return null; // TODO
+  }
+
+  public String clusterAddSlots(int... slots) {
+    return null; // TODO
+  }
+
+  public String clusterDelSlots(int... slots) {
+    return null; // TODO
+  }
+
+  public String clusterInfo() {
+    return null; // TODO
+  }
+
+  public List<String> clusterGetKeysInSlot(int slot, int count) {
+    return null; // TODO
+  }
+
+  public String clusterSetSlotNode(int slot, String nodeId) {
+    return null; // TODO
+  }
+
+  public String clusterSetSlotMigrating(int slot, String nodeId) {
+    return null; // TODO
+  }
+
+  public String clusterSetSlotImporting(int slot, String nodeId) {
+    return null; // TODO
+  }
+
+  public String clusterSetSlotStable(int slot) {
+    return null; // TODO
+  }
+
+  public String clusterForget(String nodeId) {
+    return null; // TODO
+  }
+
+  public String clusterFlushSlots() {
+    return null; // TODO
+  }
+
+  public Long clusterKeySlot(String key) {
+    return null; // TODO
+  }
+
+  public Long clusterCountKeysInSlot(int slot) {
+    return null; // TODO
+  }
+
+  public String clusterSaveConfig() {
+    return null; // TODO
+  }
+
+  public String clusterReplicate(String nodeId) {
+    return null; // TODO
+  }
+
+  public List<String> clusterSlaves(String nodeId) {
+    return null; // TODO
+  }
+
+  public String clusterFailover() {
+    return null; // TODO
+  }
+
+  public List<Object> clusterSlots() {
+    return null; // TODO
+  }
+
+  public String asking() {
+    return null; // TODO
+  }
+
+  public List<String> pubsubChannels(String pattern) {
+    return null; // TODO
+  }
+
+  public Long pubsubNumPat() {
+    return null; // TODO
+  }
+
+  public Map<String, String> pubsubNumSub(String... channels) {
+    return null; // TODO
+  }
+
+  public void close() {
+    // TODO
+  }
+
+  public void setDataSource(Pool<Jedis> jedisPool) {
+    // TODO
+  }
+
+  public Long pfadd(String key, String... elements) {
+    return null; // TODO
+  }
+
+  public long pfcount(String key) {
+    return 0L; // TODO
+  }
+
+  public long pfcount(String... keys) {
+    return 0L; // TODO
+  }
+
+  public String pfmerge(String destkey, String... sourcekeys) {
+    return null; // TODO
+  }
+
+  public MessageFuture<List<String>> blpop(int timeout, String key) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Blpop(timeout, SafeEncoder.encode(key)));
+  }
+
+  public MessageFuture<List<String>> brpop(int timeout, String key) {
+    return (MessageFuture<List<String>>) reporter.report(
+        new Brpop(timeout, SafeEncoder.encode(key)));
+  }
+
+  public Long geoadd(String key, double longitude, double latitude, String member) {
+    return null; // TODO
+  }
+
+  public Long geoadd(String key,
+      Map<String, GeoCoordinate> memberCoordinateMap) {
+    return null; // TODO
+  }
+
+  public Double geodist(String key, String member1, String member2) {
+    return null; // TODO
+  }
+
+  public Double geodist(String key, String member1, String member2, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<String> geohash(String key, String... members) {
+    return null; // TODO
+  }
+
+  public List<GeoCoordinate> geopos(String key,
+      String... members) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadius(String key, double longitude,
+      double latitude, double radius, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadius(String key, double longitude,
+      double latitude, double radius, GeoUnit unit,
+      GeoRadiusParam param) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadiusByMember(String key,
+      String member, double radius, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadiusByMember(String key,
+      String member, double radius, GeoUnit unit,
+      GeoRadiusParam param) {
+    return null; // TODO
+  }
+
+  public List<Long> bitfield(String key, String... arguments) {
+    return null; // TODO
+  }
+
+  public String ping() {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> set(byte[] key, byte[] value) {
+    return (MessageFuture<String>) reporter.report(new RedisCommand.Set(key, value, null, null, 0L));
+  }
+
+  public MessageFuture<String> set(byte[] key, byte[] value, byte[] nxxx, byte[] expx, long time) {
+    return (MessageFuture<String>) reporter.report(new RedisCommand.Set(key, value, nxxx, expx, time));
+  }
+
+  public MessageFuture<byte[]> get(byte[] key) {
+    return (MessageFuture<byte[]>) reporter.report(new Get(key));
+  }
+
+  public String quit() {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> exists(byte[]... keys) {
+    return (MessageFuture<Long>) reporter.report(new Exists(keys));
+  }
+
+  public MessageFuture<Boolean> exists(byte[] key) {
+    return transformResponse(reporter.report(new Exists(key)), BuilderFactory.BOOLEAN);
+  }
+
+  public MessageFuture<Long> del(byte[]... keys) {
+    return (MessageFuture<Long>) reporter.report(new Del(keys));
+  }
+
+  public MessageFuture<Long> del(byte[] key) {
+    return (MessageFuture<Long>) reporter.report(new Del(key));
+  }
+
+  public String type(byte[] key) {
+    return null; // TODO
+  }
+
+  public String flushDB() {
+    return null; // TODO
+  }
+
+  public Set<byte[]> keys(byte[] pattern) {
+    return null; // TODO
+  }
+
+  public byte[] randomBinaryKey() {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> rename(byte[] oldkey, byte[] newkey) {
+    return (MessageFuture<String>) reporter.report(new Rename(oldkey, newkey));
+  }
+
+  public MessageFuture<Long> renamenx(byte[] oldkey, byte[] newkey) {
+    return (MessageFuture<Long>) reporter.report(new RenameNX(oldkey, newkey));
+  }
+
+  public Long dbSize() {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> expire(byte[] key, int seconds) {
+    return (MessageFuture<Long>) reporter.report(new Expire(key, seconds));
+  }
+
+  public MessageFuture<Long> expireAt(byte[] key, long unixTime) {
+    return (MessageFuture<Long>) reporter.report(new ExpireAt(key, unixTime));
+  }
+
+  public Long ttl(byte[] key) {
+    return null; // TODO
+  }
+
+  public String select(int index) {
+    return null; // TODO
+  }
+
+  public Long move(byte[] key, int dbIndex) {
+    return null; // TODO
+  }
+
+  public String flushAll() {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> getSet(byte[] key, byte[] value) {
+    return (MessageFuture<byte[]>) reporter.report(new GetSet(key, value));
+  }
+
+  public MessageFuture<List<byte[]>> mget(byte[]... keys) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new MGet(keys));
+  }
+
+  public Long setnx(byte[] key, byte[] value) {
+    return null; // TODO
+  }
+
+  public String setex(byte[] key, int seconds, byte[] value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> mset(byte[]... keysvalues) {
+    return (MessageFuture<String>) reporter.report(new MSet(keysvalues));
+  }
+
+  public MessageFuture<Long> msetnx(byte[]... keysvalues) {
+    return (MessageFuture<Long>) reporter.report(new MSetNX(keysvalues));
+  }
+
+  public MessageFuture<Long> decrBy(byte[] key, long integer) {
+    return (MessageFuture<Long>) reporter.report(new DecrBy(key, integer));
+  }
+
+  public MessageFuture<Long> decr(byte[] key) {
+    return (MessageFuture<Long>) reporter.report(new Decr(key));
+  }
+
+  public MessageFuture<Long> incrBy(byte[] key, long integer) {
+    return (MessageFuture<Long>) reporter.report(new IncrBy(key, integer));
+  }
+
+  public Double incrByFloat(byte[] key, double integer) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> incr(byte[] key) {
+    return (MessageFuture<Long>) reporter.report(new Incr(key));
+  }
+
+  public MessageFuture<Long> append(byte[] key, byte[] value) {
+    return (MessageFuture<Long>) reporter.report(new Append(key, value));
+  }
+
+  public byte[] substr(byte[] key, int start, int end) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> hset(byte[] key, byte[] field, byte[] value) {
+    return (MessageFuture<Long>) reporter.report(new HSet(key, field, value));
+  }
+
+  public MessageFuture<byte[]> hget(byte[] key, byte[] field) {
+    return (MessageFuture<byte[]>) reporter.report(new HGet(key, field));
+  }
+
+  public MessageFuture<Long> hsetnx(byte[] key, byte[] field, byte[] value) {
+    return (MessageFuture<Long>) reporter.report(new HSetNX(key, field, value));
+  }
+
+  public MessageFuture<String> hmset(byte[] key, Map<byte[], byte[]> hash) {
+    return (MessageFuture<String>) reporter.report(new HMSet(key, hash));
+  }
+
+  public MessageFuture<List<byte[]>> hmget(byte[] key, byte[]... fields) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new HMGet(key, fields));
+  }
+
+  public MessageFuture<Long> hincrBy(byte[] key, byte[] field, long value) {
+    return (MessageFuture<Long>) reporter.report(new HIncrBy(key, field, value));
+  }
+
+  public Double hincrByFloat(byte[] key, byte[] field, double value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Boolean> hexists(byte[] key, byte[] field) {
+    return (MessageFuture<Boolean>) reporter.report(new HExists(key, field));
+  }
+
+  public MessageFuture<Long> hdel(byte[] key, byte[]... fields) {
+    return (MessageFuture<Long>) reporter.report(new HDel(key, fields));
+  }
+
+  public MessageFuture<Long> hlen(byte[] key) {
+    return (MessageFuture<Long>) reporter.report(new HLen(key));
+  }
+
+  public MessageFuture<Set<byte[]>> hkeys(byte[] key) {
+    return (MessageFuture<Set<byte[]>>) reporter.report(new HKeys(key));
+  }
+
+  public MessageFuture<List<byte[]>> hvals(byte[] key) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new HVals(key));
+  }
+
+  public MessageFuture<Map<byte[], byte[]>> hgetAll(byte[] key) {
+    return (MessageFuture<Map<byte[], byte[]>>) reporter.report(new HGetAll(key));
+  }
+
+  public Long rpush(byte[] key, byte[]... strings) {
+    return null; // TODO
+  }
+
+  public Long lpush(byte[] key, byte[]... strings) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Long> llen(byte[] key) {
+    return (MessageFuture<Long>) reporter.report(new LLen(key));
+  }
+
+  public List<byte[]> lrange(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public String ltrim(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> lindex(byte[] key, long index) {
+    return (MessageFuture<byte[]>) reporter.report(new LIndex(key, index));
+  }
+
+  public String lset(byte[] key, long index, byte[] value) {
+    return null; // TODO
+  }
+
+  public Long lrem(byte[] key, long count, byte[] value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> lpop(byte[] key) {
+    return (MessageFuture<byte[]>) reporter.report(new LPop(key));
+  }
+
+  public byte[] rpop(byte[] key) {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> rpoplpush(byte[] srckey, byte[] dstkey) {
+    return (MessageFuture<byte[]>) reporter.report(new RPopLpush(srckey, dstkey));
+  }
+
+  public Long sadd(byte[] key, byte[]... members) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> smembers(byte[] key) {
+    return null; // TODO
+  }
+
+  public Long srem(byte[] key, byte[]... member) {
+    return null; // TODO
+  }
+
+  public byte[] spop(byte[] key) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> spop(byte[] key, long count) {
+    return null; // TODO
+  }
+
+  public Long smove(byte[] srckey, byte[] dstkey, byte[] member) {
+    return null; // TODO
+  }
+
+  public Long scard(byte[] key) {
+    return null; // TODO
+  }
+
+  public Boolean sismember(byte[] key, byte[] member) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> sinter(byte[]... keys) {
+    return null; // TODO
+  }
+
+  public Long sinterstore(byte[] dstkey, byte[]... keys) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> sunion(byte[]... keys) {
+    return null; // TODO
+  }
+
+  public Long sunionstore(byte[] dstkey, byte[]... keys) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> sdiff(byte[]... keys) {
+    return null; // TODO
+  }
+
+  public Long sdiffstore(byte[] dstkey, byte[]... keys) {
+    return null; // TODO
+  }
+
+  public byte[] srandmember(byte[] key) {
+    return null; // TODO
+  }
+
+  public List<byte[]> srandmember(byte[] key, int count) {
+    return null; // TODO
+  }
+
+  public Long zadd(byte[] key, double score, byte[] member) {
+    return null; // TODO
+  }
+
+  public Long zadd(byte[] key, double score, byte[] member,
+      ZAddParams params) {
+    return null; // TODO
+  }
+
+  public Long zadd(byte[] key, Map<byte[], Double> scoreMembers) {
+    return null; // TODO
+  }
+
+  public Long zadd(byte[] key, Map<byte[], Double> scoreMembers,
+      ZAddParams params) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrange(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long zrem(byte[] key, byte[]... members) {
+    return null; // TODO
+  }
+
+  public Double zincrby(byte[] key, double score, byte[] member) {
+    return null; // TODO
+  }
+
+  public Double zincrby(byte[] key, double score, byte[] member,
+      ZIncrByParams params) {
+    return null; // TODO
+  }
+
+  public Long zrank(byte[] key, byte[] member) {
+    return null; // TODO
+  }
+
+  public Long zrevrank(byte[] key, byte[] member) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrange(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeWithScores(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeWithScores(byte[] key, long start,
+      long end) {
+    return null; // TODO
+  }
+
+  public Long zcard(byte[] key) {
+    return null; // TODO
+  }
+
+  public Double zscore(byte[] key, byte[] member) {
+    return null; // TODO
+  }
+
+  public Transaction multi() {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public List<Object> multi(TransactionBlock jedisTransaction) {
+    return null; // TODO
+  }
+
+  public void connect() {
+    // TODO
+  }
+
+  public void disconnect() {
+    // TODO
+  }
+
+  public void resetState() {
+    // TODO
+  }
+
+  public String watch(byte[]... keys) {
+    return null; // TODO
+  }
+
+  public String unwatch() {
+    return null; // TODO
+  }
+
+  public List<byte[]> sort(byte[] key) {
+    return null; // TODO
+  }
+
+  public List<byte[]> sort(byte[] key, SortingParams sortingParameters) {
+    return null; // TODO
+  }
+
+  public MessageFuture<List<byte[]>> blpop(int timeout, byte[]... keys) {
+    return transformResponse(
+        reporter.report(new Blpop(timeout, keys)), BuilderFactory.BYTE_ARRAY_LIST);
+  }
+
+  public Long sort(byte[] key, SortingParams sortingParameters, byte[] dstkey) {
+    return null; // TODO
+  }
+
+  public Long sort(byte[] key, byte[] dstkey) {
+    return null; // TODO
+  }
+
+  public MessageFuture<List<byte[]>> brpop(int timeout, byte[]... keys) {
+    return transformResponse(
+        reporter.report(new Brpop(timeout, keys)), BuilderFactory.BYTE_ARRAY_LIST);
+  }
+
+  @Deprecated
+  public MessageFuture<List<byte[]>> blpop(byte[] arg) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new Blpop(0, arg));
+  }
+
+  @Deprecated
+  public MessageFuture<List<byte[]>> brpop(byte[] arg) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new Brpop(0, arg));
+  }
+
+  public MessageFuture<List<byte[]>> blpop(byte[]... args) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new Blpop(0, args));
+  }
+
+  public MessageFuture<List<byte[]>> brpop(byte[]... args) {
+    return (MessageFuture<List<byte[]>>) reporter.report(new Brpop(0, args));
+  }
+
+  public String auth(String password) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public List<Object> pipelined(PipelineBlock jedisPipeline) {
+    return null; // TODO
+  }
+
+  public Pipeline pipelined() {
+    return null; // TODO
+  }
+
+  public Long zcount(byte[] key, double min, double max) {
+    return null; // TODO
+  }
+
+  public Long zcount(byte[] key, byte[] min, byte[] max) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByScore(byte[] key, double min, double max) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByScore(byte[] key, byte[] min, byte[] max) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByScore(byte[] key, double min, double max, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByScore(byte[] key, byte[] min, byte[] max, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(byte[] key, double min,
+      double max) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(byte[] key, byte[] min,
+      byte[] max) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(byte[] key, double min,
+      double max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrangeByScoreWithScores(byte[] key, byte[] min,
+      byte[] max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByScore(byte[] key, double max, double min) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByScore(byte[] key, byte[] max, byte[] min) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByScore(byte[] key, double max, double min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByScore(byte[] key, byte[] max, byte[] min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(byte[] key, double max,
+      double min) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(byte[] key, double max,
+      double min, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(byte[] key, byte[] max,
+      byte[] min) {
+    return null; // TODO
+  }
+
+  public Set<Tuple> zrevrangeByScoreWithScores(byte[] key, byte[] max,
+      byte[] min, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByRank(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByScore(byte[] key, double start, double end) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByScore(byte[] key, byte[] start, byte[] end) {
+    return null; // TODO
+  }
+
+  public Long zunionstore(byte[] dstkey, byte[]... sets) {
+    return null; // TODO
+  }
+
+  public Long zunionstore(byte[] dstkey, ZParams params, byte[]... sets) {
+    return null; // TODO
+  }
+
+  public Long zinterstore(byte[] dstkey, byte[]... sets) {
+    return null; // TODO
+  }
+
+  public Long zinterstore(byte[] dstkey, ZParams params, byte[]... sets) {
+    return null; // TODO
+  }
+
+  public Long zlexcount(byte[] key, byte[] min, byte[] max) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByLex(byte[] key, byte[] min, byte[] max) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrangeByLex(byte[] key, byte[] min, byte[] max, int offset, int count) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByLex(byte[] key, byte[] max, byte[] min) {
+    return null; // TODO
+  }
+
+  public Set<byte[]> zrevrangeByLex(byte[] key, byte[] max, byte[] min, int offset,
+      int count) {
+    return null; // TODO
+  }
+
+  public Long zremrangeByLex(byte[] key, byte[] min, byte[] max) {
+    return null; // TODO
+  }
+
+  public String save() {
+    return null; // TODO
+  }
+
+  public String bgsave() {
+    return null; // TODO
+  }
+
+  public String bgrewriteaof() {
+    return null; // TODO
+  }
+
+  public Long lastsave() {
+    return null; // TODO
+  }
+
+  public String shutdown() {
+    return null; // TODO
+  }
+
+  public String info() {
+    return null; // TODO
+  }
+
+  public String info(String section) {
+    return null; // TODO
+  }
+
+  public void monitor(JedisMonitor jedisMonitor) {
+    // TODO
+  }
+
+  public String slaveof(String host, int port) {
+    return null; // TODO
+  }
+
+  public String slaveofNoOne() {
+    return null; // TODO
+  }
+
+  public List<byte[]> configGet(byte[] pattern) {
+    return null; // TODO
+  }
+
+  public String configResetStat() {
+    return null; // TODO
+  }
+
+  public byte[] configSet(byte[] parameter, byte[] value) {
+    return null; // TODO
+  }
+
+  public boolean isConnected() {
+    return false; // TODO
+  }
+
+  public Long strlen(byte[] key) {
+    return null; // TODO
+  }
+
+  public void sync() {
+    // TODO
+  }
+
+  public Long lpushx(byte[] key, byte[]... string) {
+    return null; // TODO
+  }
+
+  public Long persist(byte[] key) {
+    return null; // TODO
+  }
+
+  public Long rpushx(byte[] key, byte[]... string) {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> echo(byte[] string) {
+    return (MessageFuture<byte[]>) reporter.report(new Echo(string));
+  }
+
+  public MessageFuture<Long> linsert(byte[] key, LIST_POSITION where, byte[] pivot,
+      byte[] value) {
+    return (MessageFuture<Long>) reporter.report(new LInsert(key, where, pivot, value));
+  }
+
+  public String debug(DebugParams params) {
+    return null; // TODO
+  }
+
+  public Client getClient() {
+    return null; // TODO
+  }
+
+  public byte[] brpoplpush(byte[] source, byte[] destination, int timeout) {
+    return null; // TODO
+  }
+
+  public Boolean setbit(byte[] key, long offset, boolean value) {
+    return null; // TODO
+  }
+
+  public Boolean setbit(byte[] key, long offset, byte[] value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<Boolean> getbit(byte[] key, long offset) {
+    return (MessageFuture<Boolean>) reporter.report(new GetBit(key, offset));
+  }
+
+  public MessageFuture<Long> bitpos(byte[] key, boolean value) {
+    return (MessageFuture<Long>) reporter.report(new BitPos(key, value, new BitPosParams()));
+  }
+
+  public MessageFuture<Long> bitpos(byte[] key, boolean value, redis.clients.jedis.BitPosParams params) {
+    return (MessageFuture<Long>) reporter.report(new BitPos(key, value, params));
+  }
+
+  public Long setrange(byte[] key, long offset, byte[] value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<byte[]> getrange(byte[] key, long startOffset, long endOffset) {
+    return transformResponse(reporter.report(
+        new GetRange(SafeEncoder.encode(key), startOffset, endOffset)), BuilderFactory.BYTE_ARRAY);
+  }
+
+  public Long publish(byte[] channel, byte[] message) {
+    return null; // TODO
+  }
+
+  public void subscribe(BinaryJedisPubSub jedisPubSub, byte[]... channels) {
+    // TODO
+  }
+
+  public void psubscribe(BinaryJedisPubSub jedisPubSub, byte[]... patterns) {
+    // TODO
+  }
+
+  public Long getDB() {
+    return null; // TODO
+  }
+
+  public Object eval(byte[] script, List<byte[]> keys, List<byte[]> args) {
+    return null; // TODO
+  }
+
+  public Object eval(byte[] script, byte[] keyCount, byte[]... params) {
+    return null; // TODO
+  }
+
+  public Object eval(byte[] script, int keyCount, byte[]... params) {
+    return null; // TODO
+  }
+
+  public Object eval(byte[] script) {
+    return null; // TODO
+  }
+
+  public Object evalsha(byte[] sha1) {
+    return null; // TODO
+  }
+
+  public Object evalsha(byte[] sha1, List<byte[]> keys, List<byte[]> args) {
+    return null; // TODO
+  }
+
+  public Object evalsha(byte[] sha1, int keyCount, byte[]... params) {
+    return null; // TODO
+  }
+
+  public String scriptFlush() {
+    return null; // TODO
+  }
+
+  public Long scriptExists(byte[] sha1) {
+    return null; // TODO
+  }
+
+  public List<Long> scriptExists(byte[]... sha1) {
+    return null; // TODO
+  }
+
+  public byte[] scriptLoad(byte[] script) {
+    return null; // TODO
+  }
+
+  public String scriptKill() {
+    return null; // TODO
+  }
+
+  public String slowlogReset() {
+    return null; // TODO
+  }
+
+  public Long slowlogLen() {
+    return null; // TODO
+  }
+
+  public List<byte[]> slowlogGetBinary() {
+    return null; // TODO
+  }
+
+  public List<byte[]> slowlogGetBinary(long entries) {
+    return null; // TODO
+  }
+
+  public Long objectRefcount(byte[] key) {
+    return null; // TODO
+  }
+
+  public byte[] objectEncoding(byte[] key) {
+    return null; // TODO
+  }
+
+  public Long objectIdletime(byte[] key) {
+    return null; // TODO
+  }
+
+  public Long bitcount(byte[] key) {
+    return null; // TODO
+  }
+
+  public Long bitcount(byte[] key, long start, long end) {
+    return null; // TODO
+  }
+
+  public Long bitop(BitOP op, byte[] destKey, byte[]... srcKeys) {
+    return null; // TODO
+  }
+
+  public byte[] dump(byte[] key) {
+    return null; // TODO
+  }
+
+  public String restore(byte[] key, int ttl, byte[] serializedValue) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public Long pexpire(byte[] key, int milliseconds) {
+    return null; // TODO
+  }
+
+  public Long pexpire(byte[] key, long milliseconds) {
+    return null; // TODO
+  }
+
+  public Long pexpireAt(byte[] key, long millisecondsTimestamp) {
+    return null; // TODO
+  }
+
+  public Long pttl(byte[] key) {
+    return null; // TODO
+  }
+
+  @Deprecated
+  public String psetex(byte[] key, int milliseconds, byte[] value) {
+    return null; // TODO
+  }
+
+  public String psetex(byte[] key, long milliseconds, byte[] value) {
+    return null; // TODO
+  }
+
+  public MessageFuture<String> set(byte[] key, byte[] value, byte[] nxxx) {
+    return (MessageFuture<String>) reporter.report(new RedisCommand.Set(key, value, nxxx, null, 0L));
+  }
+
+  public MessageFuture<String> set(byte[] key, byte[] value, byte[] nxxx, byte[] expx, int time) {
+    return (MessageFuture<String>) reporter.report(new RedisCommand.Set(key, value, nxxx, expx, time));
+  }
+
+  public String clientKill(byte[] client) {
+    return null; // TODO
+  }
+
+  public String clientGetname() {
+    return null; // TODO
+  }
+
+  public String clientList() {
+    return null; // TODO
+  }
+
+  public String clientSetname(byte[] name) {
+    return null; // TODO
+  }
+
+  public List<String> time() {
+    return null; // TODO
+  }
+
+  public String migrate(byte[] host, int port, byte[] key, int destinationDb, int timeout) {
+    return null; // TODO
+  }
+
+  public Long waitReplicas(int replicas, long timeout) {
+    return null; // TODO
+  }
+
+  public Long pfadd(byte[] key, byte[]... elements) {
+    return null; // TODO
+  }
+
+  public long pfcount(byte[] key) {
+    return 0L; // TODO
+  }
+
+  public String pfmerge(byte[] destkey, byte[]... sourcekeys) {
+    return null; // TODO
+  }
+
+  public Long pfcount(byte[]... keys) {
+    return null; // TODO
+  }
+
+  public ScanResult<byte[]> scan(byte[] cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<byte[]> scan(byte[] cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<Entry<byte[], byte[]>> hscan(byte[] key,
+      byte[] cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<Entry<byte[], byte[]>> hscan(byte[] key,
+      byte[] cursor, ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<byte[]> sscan(byte[] key, byte[] cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<byte[]> sscan(byte[] key, byte[] cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public ScanResult<Tuple> zscan(byte[] key, byte[] cursor) {
+    return null; // TODO
+  }
+
+  public ScanResult<Tuple> zscan(byte[] key, byte[] cursor,
+      ScanParams params) {
+    return null; // TODO
+  }
+
+  public Long geoadd(byte[] key, double longitude, double latitude, byte[] member) {
+    return null; // TODO
+  }
+
+  public Long geoadd(byte[] key,
+      Map<byte[], GeoCoordinate> memberCoordinateMap) {
+    return null; // TODO
+  }
+
+  public Double geodist(byte[] key, byte[] member1, byte[] member2) {
+    return null; // TODO
+  }
+
+  public Double geodist(byte[] key, byte[] member1, byte[] member2, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<byte[]> geohash(byte[] key, byte[]... members) {
+    return null; // TODO
+  }
+
+  public List<GeoCoordinate> geopos(byte[] key, byte[]... members) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadius(byte[] key, double longitude,
+      double latitude, double radius, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadius(byte[] key, double longitude,
+      double latitude, double radius, GeoUnit unit,
+      GeoRadiusParam param) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadiusByMember(byte[] key,
+      byte[] member, double radius, GeoUnit unit) {
+    return null; // TODO
+  }
+
+  public List<GeoRadiusResponse> georadiusByMember(byte[] key,
+      byte[] member, double radius, GeoUnit unit,
+      GeoRadiusParam param) {
+    return null; // TODO
+  }
+
+  public List<byte[]> bitfield(byte[] key, byte[]... arguments) {
+    return null; // TODO
+  }
+
+  public JedisPool unwrapped() {
+    return jedisPool;
+  }
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BitPosParams.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BitPosParams.java
new file mode 100644
index 0000000..e1003b1
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BitPosParams.java
@@ -0,0 +1,11 @@
+package io.github.tramchamploo.bufferslayer;
+
+/**
+ * This opens no-arg constructor on {@link redis.clients.jedis.BitPosParams}
+ */
+@SuppressWarnings("ClassNameSameAsAncestorName")
+class BitPosParams extends redis.clients.jedis.BitPosParams {
+  BitPosParams() {
+    super();
+  }
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/Builder.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/Builder.java
new file mode 100644
index 0000000..bd5342d
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/Builder.java
@@ -0,0 +1,10 @@
+package io.github.tramchamploo.bufferslayer;
+
+/**
+ * Builder that build response to type T
+ * @param <T> the result type
+ */
+abstract class Builder<T> {
+
+  abstract T build(Object data);
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BuilderFactory.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BuilderFactory.java
new file mode 100644
index 0000000..0e61385
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/BuilderFactory.java
@@ -0,0 +1,109 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import redis.clients.util.SafeEncoder;
+
+/**
+ * Factory for {@link Builder} that builds result
+ */
+final class BuilderFactory {
+
+  @SuppressWarnings("unchecked")
+  static final Builder<List<byte[]>> BYTE_ARRAY_LIST = new Builder<List<byte[]>>() {
+    @Override
+    List<byte[]> build(Object data) {
+      if (null == data) {
+        return Collections.emptyList();
+      }
+      List<String> l = (List<String>) data;
+      final ArrayList<byte[]> result = new ArrayList<>(l.size());
+      for (final String s: l) {
+        if (s == null) {
+          result.add(null);
+        } else {
+          result.add(SafeEncoder.encode(s));
+        }
+      }
+      return result;
+    }
+  };
+
+  static final Builder<String> STRING = new Builder<String>() {
+    public String build(Object data) {
+      return data == null ? null : SafeEncoder.encode((byte[]) data);
+    }
+  };
+
+  static final Builder<Boolean> BOOLEAN = new Builder<Boolean>() {
+    public Boolean build(Object data) {
+      return ((Long) data) == 1;
+    }
+  };
+
+  static final Builder<List<String>> STRING_LIST = new Builder<List<String>>() {
+    @SuppressWarnings("unchecked")
+    public List<String> build(Object data) {
+      if (null == data) {
+        return Collections.emptyList();
+      }
+      List<byte[]> l = (List<byte[]>) data;
+      final ArrayList<String> result = new ArrayList<String>(l.size());
+      for (final byte[] barray : l) {
+        if (barray == null) {
+          result.add(null);
+        } else {
+          result.add(SafeEncoder.encode(barray));
+        }
+      }
+      return result;
+    }
+  };
+
+  public static final Builder<byte[]> BYTE_ARRAY = new Builder<byte[]>() {
+    public byte[] build(Object data) {
+      return SafeEncoder.encode((String) data);
+    }
+  };
+
+  public static final Builder<Map<String, String>> STRING_MAP = new Builder<Map<String, String>>() {
+    @SuppressWarnings("unchecked")
+    public Map<String, String> build(Object data) {
+      final Map<byte[], byte[]> byteMap = (Map<byte[], byte[]>) data;
+      final Map<String, String> hash = new HashMap<>(byteMap.size(), 1);
+      Iterator<Entry<byte[], byte[]>> iterator = byteMap.entrySet().iterator();
+      while (iterator.hasNext()) {
+        Entry<byte[], byte[]> entry = iterator.next();
+        hash.put(SafeEncoder.encode(entry.getKey()), SafeEncoder.encode(entry.getValue()));
+      }
+
+      return hash;
+    }
+  };
+
+  public static final Builder<Set<String>> STRING_SET = new Builder<Set<String>>() {
+    @SuppressWarnings("unchecked")
+    public Set<String> build(Object data) {
+      if (null == data) {
+        return Collections.emptySet();
+      }
+      Set<byte[]> l = (Set<byte[]>) data;
+      final Set<String> result = new HashSet<>(l.size());
+      for (final byte[] barray : l) {
+        if (barray == null) {
+          result.add(null);
+        } else {
+          result.add(SafeEncoder.encode(barray));
+        }
+      }
+      return result;
+    }
+  };
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/JedisSender.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/JedisSender.java
new file mode 100644
index 0000000..dd8dd8f
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/JedisSender.java
@@ -0,0 +1,63 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.List;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.Pipeline;
+
+/**
+ * Send buffered commands in jedis's pipeline
+ */
+final class JedisSender implements Sender<RedisCommand, Object> {
+
+  private final JedisPool jedisPool;
+
+  JedisSender(JedisPool jedisPool) {
+    this.jedisPool = jedisPool;
+  }
+
+  @Override
+  public CheckResult check() {
+    Jedis jedis = null;
+    try {
+      jedis = jedisPool.getResource();
+
+      String ping = jedis.ping();
+      if ("PONG".equalsIgnoreCase(ping)) {
+        return CheckResult.OK;
+      }
+
+      return CheckResult.failed(new RuntimeException("PING doesn't get PONG."));
+    } catch (Exception e) {
+      return CheckResult.failed(e);
+    } finally {
+      if (jedis != null) {
+        jedis.close();
+      }
+    }
+  }
+
+  @Override
+  public void close() {
+    jedisPool.close();
+  }
+
+  @Override
+  public List<Object> send(List<RedisCommand> messages) {
+    Jedis jedis = null;
+    try {
+      jedis = jedisPool.getResource();
+
+      Pipeline pipeline = jedis.pipelined();
+      for (RedisCommand command : messages) {
+        command.apply(pipeline);
+      }
+
+      return pipeline.syncAndReturnAll();
+    } finally {
+      if (jedis != null) {
+        jedis.close();
+      }
+    }
+  }
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/MultiKeyRedisCommand.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/MultiKeyRedisCommand.java
new file mode 100644
index 0000000..3a9af7a
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/MultiKeyRedisCommand.java
@@ -0,0 +1,155 @@
+package io.github.tramchamploo.bufferslayer;
+
+import redis.clients.jedis.Pipeline;
+
+/**
+ * A redis command with multi keys
+ */
+abstract class MultiKeyRedisCommand extends RedisCommand {
+
+    final byte[][] keys;
+
+    MultiKeyRedisCommand(byte[][] keys) {
+      super(keys[0]);
+      this.keys = keys;
+    }
+
+    String keysString() {
+      return RedisCommandUtil.toString(keys);
+    }
+
+  // Commands start
+
+  final static class Blpop extends MultiKeyRedisCommand {
+
+    final int timeout;
+
+    Blpop(int timeout, byte[]... keys) {
+      super(keys);
+      this.timeout = timeout;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      if (timeout == 0) {
+        pipeline.blpop(keys);
+      } else {
+        pipeline.blpop(timeout, keys);
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "Blpop(" + keysString() + ", " + timeout + ")";
+    }
+  }
+
+  final static class Brpop extends MultiKeyRedisCommand {
+
+    final int timeout;
+
+    Brpop(int timeout, byte[]... keys) {
+      super(keys);
+      this.timeout = timeout;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      if (timeout == 0) {
+        pipeline.brpop(keys);
+      } else {
+        pipeline.brpop(timeout, keys);
+      }
+    }
+
+    @Override
+    public String toString() {
+      return "Brpop(" + keysString() + ", " + timeout + ")";
+    }
+  }
+
+  final static class Del extends MultiKeyRedisCommand {
+
+    Del(byte[]... keys) {
+      super(keys);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.del(keys);
+    }
+
+    @Override
+    public String toString() {
+      return "Del(" + keysString() + ")";
+    }
+  }
+
+  final static class Exists extends MultiKeyRedisCommand {
+
+    Exists(byte[]... keys) {
+      super(keys);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.exists(keys);
+    }
+
+    @Override
+    public String toString() {
+      return "Exists(" + keysString() + ")";
+    }
+  }
+
+  final static class MGet extends MultiKeyRedisCommand {
+
+    MGet(byte[]... keys) {
+      super(keys);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.mget(keys);
+    }
+
+    @Override
+    public String toString() {
+      return "MGet(" + keysString() + ")";
+    }
+  }
+
+  final static class MSet extends MultiKeyRedisCommand {
+
+    MSet(byte[]... keysvalues) {
+      super(keysvalues);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.mset(keys);
+    }
+
+    @Override
+    public String toString() {
+      return "MSet(" + keysString() + ")";
+    }
+  }
+
+  final static class MSetNX extends MultiKeyRedisCommand {
+
+    MSetNX(byte[]... keysvalues) {
+      super(keysvalues);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.msetnx(keys);
+    }
+
+    @Override
+    public String toString() {
+      return "MSetNX(" + keysString() + ")";
+    }
+  }
+}
\ No newline at end of file
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommand.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommand.java
new file mode 100644
index 0000000..d9955f7
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommand.java
@@ -0,0 +1,713 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.Map;
+import redis.clients.jedis.BinaryClient.LIST_POSITION;
+import redis.clients.jedis.Pipeline;
+import redis.clients.util.SafeEncoder;
+
+/**
+ * A redis command to be executed
+ */
+abstract class RedisCommand extends Message {
+
+  final byte[] key;
+
+  RedisCommand(byte[] key) {
+    this.key = key;
+  }
+
+  /**
+   * This implies how redis pipeline execute this command.
+   *
+   * @param pipeline pipeline to behave on.
+   */
+  protected abstract void apply(Pipeline pipeline);
+
+  @Override
+  public MessageKey asMessageKey() {
+    return Message.SINGLE_KEY;
+  }
+
+  String keysString() {
+    return SafeEncoder.encode(key);
+  }
+
+  // Commands start
+
+  final static class Append extends RedisCommand {
+
+    final byte[] value;
+
+    Append(byte[] key, byte[] value) {
+      super(key);
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.append(key, value);
+    }
+
+    @Override
+    public String toString() {
+      return "Append(" + new String(key) + ", " + SafeEncoder.encode(value) + ")";
+    }
+  }
+
+  final static class Decr extends RedisCommand {
+
+    Decr(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.decr(key);
+    }
+
+    @Override
+    public String toString() {
+      return "Decr(" + keysString() + ")";
+    }
+  }
+
+  final static class DecrBy extends RedisCommand {
+
+    final long value;
+
+    DecrBy(byte[] key, long value) {
+      super(key);
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.decrBy(key, value);
+    }
+
+    @Override
+    public String toString() {
+      return "DecrBy(" + keysString() + ", " +  + value + ")";
+    }
+  }
+
+  final static class Echo extends RedisCommand {
+
+    Echo(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.echo(key);
+    }
+
+    @Override
+    public String toString() {
+      return "Echo(" + keysString() + ")";
+    }
+  }
+
+  final static class Expire extends RedisCommand {
+
+    final int seconds;
+
+    Expire(byte[] key, int seconds) {
+      super(key);
+      this.seconds = seconds;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.expire(key, seconds);
+    }
+
+    @Override
+    public String toString() {
+      return "Expire(" + keysString() + ", " + seconds + ")";
+    }
+  }
+
+  final static class ExpireAt extends RedisCommand {
+
+    final long unixTime;
+
+    ExpireAt(byte[] key, long unixTime) {
+      super(key);
+      this.unixTime = unixTime;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.expireAt(key, unixTime);
+    }
+
+    @Override
+    public String toString() {
+      return "ExpireAt(" + keysString() + ", " + unixTime + ")";
+    }
+  }
+
+  final static class Get extends RedisCommand {
+
+    Get(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.get(key);
+    }
+
+    @Override
+    public String toString() {
+      return "Get(" + keysString() + ")";
+    }
+  }
+
+  final static class GetBit extends RedisCommand {
+
+    long offset;
+
+    GetBit(byte[] key, long offset) {
+      super(key);
+      this.offset = offset;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.getbit(key, offset);
+    }
+
+    @Override
+    public String toString() {
+      return "GetBit(" + keysString() + ", " + offset + ")";
+    }
+  }
+
+  final static class BitPos extends RedisCommand {
+
+    boolean value;
+    redis.clients.jedis.BitPosParams params;
+
+    BitPos(byte[] key, boolean value, redis.clients.jedis.BitPosParams params) {
+      super(key);
+      this.value = value;
+      this.params = params;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.bitpos(key, value, params);
+    }
+
+    @Override
+    public String toString() {
+      return "BitPos(" + keysString() + ", " + value + ", "
+          + BuilderFactory.STRING_LIST.build(params.getParams()).toString() + ")";
+    }
+  }
+
+  final static class GetRange extends RedisCommand {
+
+    String keyString;
+    long startOffset;
+    long endOffset;
+
+    GetRange(String key, long startOffset, long endOffset) {
+      super(null);
+      this.keyString = key;
+      this.startOffset = startOffset;
+      this.endOffset = endOffset;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.getrange(keyString, startOffset, endOffset);
+    }
+
+    @Override
+    public String toString() {
+      return "GetRange(" + keysString() + ", " + startOffset + ", " + endOffset + ")";
+    }
+
+    String keysString() {
+      return keyString;
+    }
+  }
+
+  final static class Set extends RedisCommand {
+
+    byte[] value;
+    byte[] nxxx;
+    byte[] expx;
+    long time;
+
+    Set(byte[] key, byte[] value, byte[] nxxx, byte[] expx, long time) {
+      super(key);
+      this.value = value;
+      this.nxxx = nxxx;
+      this.expx = expx;
+      this.time = time;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      if (expx != null) {
+        pipeline.set(key, value, nxxx, expx, (int) time);
+      } else if (nxxx != null) {
+        pipeline.set(key, value, nxxx);
+      } else {
+        pipeline.set(key, value);
+      }
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("Set(")
+          .append(keysString()).append(", ")
+          .append(SafeEncoder.encode(value));
+      if (nxxx != null) {
+        builder.append(", ").append(SafeEncoder.encode(nxxx));
+      }
+      if (expx != null) {
+        builder.append(", ").append(SafeEncoder.encode(expx))
+            .append(", ").append(time);
+      }
+      return builder.append(")").toString();
+    }
+  }
+
+  final static class GetSet extends RedisCommand {
+
+    byte[] value;
+
+    GetSet(byte[] key, byte[] value) {
+      super(key);
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.getSet(key, value);
+    }
+
+    @Override
+    public String toString() {
+      return "GetSet(" + keysString() + ", " + SafeEncoder.encode(value) + ")";
+    }
+  }
+
+  final static class HDel extends RedisCommand {
+
+    final byte[][] field;
+
+    HDel(byte[] key, byte[]... field) {
+      super(key);
+      this.field = field;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hdel(key, field);
+    }
+
+    @Override
+    public String toString() {
+        return "HDel(" + keysString() + ", " + RedisCommandUtil.toString(field) + ")";
+    }
+  }
+
+  final static class HExists extends RedisCommand {
+
+    final byte[] field;
+
+    HExists(byte[] key, byte[] field) {
+      super(key);
+      this.field = field;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hexists(key, field);
+    }
+
+    @Override
+    public String toString() {
+      return "HExists(" + keysString() + ", " + SafeEncoder.encode(field) + ")";
+    }
+  }
+
+  final static class HGet extends RedisCommand {
+
+    final byte[] field;
+
+    HGet(byte[] key, byte[] field) {
+      super(key);
+      this.field = field;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hget(key, field);
+    }
+
+    @Override
+    public String toString() {
+      return "HGet(" + keysString() + ", " + SafeEncoder.encode(field) + ")";
+    }
+  }
+
+  final static class HGetAll extends RedisCommand {
+
+    HGetAll(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hgetAll(key);
+    }
+
+    @Override
+    public String toString() {
+      return "HGetAll(" + keysString() + ")";
+    }
+  }
+
+  final static class HIncrBy extends RedisCommand {
+
+    final byte[] field;
+    final long value;
+
+    HIncrBy(byte[] key, byte[] field, long value) {
+      super(key);
+      this.field = field;
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hincrBy(key, field, value);
+    }
+
+    @Override
+    public String toString() {
+      return "HIncrBy(" + keysString() + ", " + SafeEncoder.encode(field) + ", " + value + ")";
+    }
+  }
+
+  final static class HKeys extends RedisCommand {
+
+    HKeys(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hkeys(key);
+    }
+
+    @Override
+    public String toString() {
+      return "HKeys(" + keysString() + ")";
+    }
+  }
+
+  final static class HLen extends RedisCommand {
+
+    HLen(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hlen(key);
+    }
+
+    @Override
+    public String toString() {
+      return "HLen(" + keysString() + ")";
+    }
+  }
+
+  final static class HMGet extends RedisCommand {
+
+    final byte[][] field;
+
+    HMGet(byte[] key, byte[]... field) {
+      super(key);
+      this.field = field;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hmget(key, field);
+    }
+
+    @Override
+    public String toString() {
+      return "HMGet(" + keysString() + ", " + RedisCommandUtil.toString(field) + ")";
+    }
+  }
+
+  final static class HMSet extends RedisCommand {
+
+    final Map<byte[], byte[]> hash;
+
+    HMSet(byte[] key, Map<byte[], byte[]> hash) {
+      super(key);
+      this.hash = hash;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hmset(key, hash);
+    }
+
+    @Override
+    public String toString() {
+      return "HMSet(" + keysString() + ", " + RedisCommandUtil.toString(hash) + ")";
+    }
+  }
+
+  final static class HSet extends RedisCommand {
+
+    final byte[] field;
+    final byte[] value;
+
+    HSet(byte[] key, byte[] field, byte[] value) {
+      super(key);
+      this.field = field;
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hset(key, field, value);
+    }
+
+    @Override
+    public String toString() {
+      return "HSet(" + keysString() + ", " + SafeEncoder.encode(field) + ", "
+          + SafeEncoder.encode(value) + ")";
+    }
+  }
+
+  final static class HSetNX extends RedisCommand {
+
+    final byte[] field;
+    final byte[] value;
+
+    HSetNX(byte[] key, byte[] field, byte[] value) {
+      super(key);
+      this.field = field;
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hsetnx(key, field, value);
+    }
+
+    @Override
+    public String toString() {
+      return "HSetNX(" + keysString() + ", " + SafeEncoder.encode(field) + ", "
+          + SafeEncoder.encode(value) + ")";
+    }
+  }
+
+  final static class HVals extends RedisCommand {
+
+    HVals(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.hvals(key);
+    }
+
+    @Override
+    public String toString() {
+      return "HVals(" + keysString() + ")";
+    }
+  }
+
+  final static class Incr extends RedisCommand {
+
+    Incr(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.incr(key);
+    }
+
+    @Override
+    public String toString() {
+      return "Incr(" + keysString() + ")";
+    }
+  }
+
+  final static class IncrBy extends RedisCommand {
+
+    final long value;
+
+    IncrBy(byte[] key, long value) {
+      super(key);
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.incrBy(key, value);
+    }
+
+    @Override
+    public String toString() {
+      return "IncrBy(" + keysString() + ", " + value + ")";
+    }
+  }
+
+  final static class Rename extends RedisCommand {
+
+    final byte[] newKey;
+
+    Rename(byte[] oldKey, byte[] newKey) {
+      super(oldKey);
+      this.newKey = newKey;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.rename(key, newKey);
+    }
+
+    @Override
+    public String toString() {
+      return "Rename(" + keysString() + ", " + newKey + ")";
+    }
+  }
+
+  final static class RenameNX extends RedisCommand {
+
+    final byte[] newKey;
+
+    RenameNX(byte[] oldKey, byte[] newKey) {
+      super(oldKey);
+      this.newKey = newKey;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.renamenx(key, newKey);
+    }
+
+    @Override
+    public String toString() {
+      return "RenameNX(" + keysString() + ", " + newKey + ")";
+    }
+  }
+
+  final static class RPopLpush extends RedisCommand {
+
+    final byte[] dstkey;
+
+    RPopLpush(byte[] srckey, byte[] dstkey) {
+      super(srckey);
+      this.dstkey = dstkey;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.rpoplpush(key, dstkey);
+    }
+
+    @Override
+    public String toString() {
+      return "RPopLpush(" + keysString() + ", " + SafeEncoder.encode(dstkey) + ")";
+    }
+  }
+
+  final static class LIndex extends RedisCommand {
+
+    final long index;
+
+    LIndex(byte[] key, long index) {
+      super(key);
+      this.index = index;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.lindex(key, index);
+    }
+
+    @Override
+    public String toString() {
+      return "LIndex(" + keysString() + ", " + index + ")";
+    }
+  }
+
+  final static class LInsert extends RedisCommand {
+
+    final LIST_POSITION where;
+    final byte[] pivot;
+    final byte[] value;
+
+    LInsert(byte[] key, LIST_POSITION where, byte[] pivot, byte[] value) {
+      super(key);
+      this.where = where;
+      this.pivot = pivot;
+      this.value = value;
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.linsert(key, where, pivot, value);
+    }
+
+    @Override
+    public String toString() {
+      return "LInsert(" + keysString() + ", " + where + ", " +
+          SafeEncoder.encode(pivot) + ", " + SafeEncoder.encode(value) + ")";
+    }
+  }
+
+  final static class LLen extends RedisCommand {
+
+    LLen(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.llen(key);
+    }
+
+    @Override
+    public String toString() {
+      return "LLen(" + keysString() + ")";
+    }
+  }
+
+  final static class LPop extends RedisCommand {
+
+    LPop(byte[] key) {
+      super(key);
+    }
+
+    @Override
+    protected void apply(Pipeline pipeline) {
+      pipeline.lpop(key);
+    }
+
+    @Override
+    public String toString() {
+      return "LPop(" + keysString() + ")";
+    }
+  }
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommandUtil.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommandUtil.java
new file mode 100644
index 0000000..4316013
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/RedisCommandUtil.java
@@ -0,0 +1,40 @@
+package io.github.tramchamploo.bufferslayer;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import redis.clients.util.SafeEncoder;
+
+class RedisCommandUtil {
+
+  static String toString(byte[][] bArray) {
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < bArray.length; i++) {
+      if (i > 0) {
+        builder.append(", ");
+      }
+      builder.append(SafeEncoder.encode(bArray[i]));
+    }
+    return builder.toString();
+  }
+
+  static String toString(Map<byte[], byte[]> bMap) {
+    StringBuilder builder = new StringBuilder();
+    Iterator<Entry<byte[], byte[]>> iterator = bMap.entrySet().iterator();
+    int count = 0;
+
+    while (iterator.hasNext()) {
+      Entry<byte[], byte[]> entry = iterator.next();
+      if (count > 0) {
+        builder.append(", ");
+      }
+
+      builder.append(SafeEncoder.encode(entry.getKey()))
+        .append("=")
+        .append(SafeEncoder.encode(entry.getValue()));
+
+      count++;
+    }
+    return builder.toString();
+  }
+}
diff --git a/jedis/src/main/java/io/github/tramchamploo/bufferslayer/ResponseUtil.java b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/ResponseUtil.java
new file mode 100644
index 0000000..2a21b4c
--- /dev/null
+++ b/jedis/src/main/java/io/github/tramchamploo/bufferslayer/ResponseUtil.java
@@ -0,0 +1,27 @@
+package io.github.tramchamploo.bufferslayer;
+
+import io.github.tramchamploo.bufferslayer.internal.DefaultMessagePromise;
+import io.github.tramchamploo.bufferslayer.internal.Future;
+import io.github.tramchamploo.bufferslayer.internal.FutureListener;
+import io.github.tramchamploo.bufferslayer.internal.MessageFuture;
+
+final class ResponseUtil {
+
+  @SuppressWarnings("unchecked")
+  static <T> MessageFuture<T> transformResponse(MessageFuture<?> future, final Builder<T> builder) {
+    final DefaultMessagePromise<T> promise = new DefaultMessagePromise<>(future.message());
+    future.addListener(new FutureListener<Object>() {
+
+      @Override
+      public void operationComplete(Future<Object> future) throws Exception {
+        if (future.isSuccess()) {
+          promise.setSuccess(builder.build(future.get()));
+        } else {
+          promise.setFailure(future.cause());
+        }
+      }
+    });
+
+    return promise;
+  }
+}
diff --git a/jedis/src/test/java/io/github/tramchamploo/bufferslayer/AssertUtil.java b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/AssertUtil.java
new file mode 100644
index 0000000..eec45dd
--- /dev/null
+++ b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/AssertUtil.java
@@ -0,0 +1,49 @@
+package io.github.tramchamploo.bufferslayer;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.junit.ComparisonFailure;
+
+public class AssertUtil {
+
+  static void assertByteArrayListEquals(List<byte[]> expected, List<byte[]> actual) {
+    assertEquals(expected.size(), actual.size());
+    for (int n = 0; n < expected.size(); n++) {
+      assertArrayEquals(expected.get(n), actual.get(n));
+    }
+  }
+
+  static void assertByteArraySetEquals(Set<byte[]> expected, Set<byte[]> actual) {
+    assertEquals(expected.size(), actual.size());
+    Iterator<byte[]> e = expected.iterator();
+    while (e.hasNext()) {
+      byte[] next = e.next();
+      boolean contained = false;
+      for (byte[] element : expected) {
+        if (Arrays.equals(next, element)) {
+          contained = true;
+        }
+      }
+      if (!contained) {
+        throw new ComparisonFailure("element is missing", Arrays.toString(next), actual.toString());
+      }
+    }
+  }
+
+  static boolean arrayContains(List<byte[]> array, byte[] expected) {
+    for (byte[] a : array) {
+      try {
+        assertArrayEquals(a, expected);
+        return true;
+      } catch (AssertionError e) {
+
+      }
+    }
+    return false;
+  }
+}
diff --git a/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisListCommandsTest.java b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisListCommandsTest.java
new file mode 100644
index 0000000..17a126f
--- /dev/null
+++ b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisListCommandsTest.java
@@ -0,0 +1,207 @@
+package io.github.tramchamploo.bufferslayer;
+
+import static io.github.tramchamploo.bufferslayer.AssertUtil.assertByteArrayListEquals;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import io.github.tramchamploo.bufferslayer.internal.MessageFuture;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import org.junit.Test;
+import redis.clients.jedis.BinaryClient.LIST_POSITION;
+
+public class BatchJedisListCommandsTest extends BatchJedisTestBase {
+
+  final byte[] bA = { 0x0A };
+  final byte[] bB = { 0x0B };
+  final byte[] bC = { 0x0C };
+  final byte[] bdst = { 0x11, 0x12, 0x13, 0x14 };
+  final byte[] b1 = { 0x01 };
+  final byte[] b2 = { 0x02 };
+  final byte[] b3 = { 0x03 };
+
+  @Test
+  public void rpoplpush() {
+    jedis.rpush("foo", "a");
+    jedis.rpush("foo", "b");
+    jedis.rpush("foo", "c");
+
+    jedis.rpush("dst", "foo");
+    jedis.rpush("dst", "bar");
+
+    String element = blocking(batchJedis.rpoplpush("foo", "dst"));
+
+    assertEquals("c", element);
+
+    List<String> srcExpected = new ArrayList<String>();
+    srcExpected.add("a");
+    srcExpected.add("b");
+
+    List<String> dstExpected = new ArrayList<String>();
+    dstExpected.add("c");
+    dstExpected.add("foo");
+    dstExpected.add("bar");
+
+    assertEquals(srcExpected, jedis.lrange("foo", 0, 1000));
+    assertEquals(dstExpected, jedis.lrange("dst", 0, 1000));
+
+    // Binary
+    jedis.rpush(bfoo, bA);
+    jedis.rpush(bfoo, bB);
+    jedis.rpush(bfoo, bC);
+
+    jedis.rpush(bdst, bfoo);
+    jedis.rpush(bdst, bbar);
+
+    byte[] belement = blocking(batchJedis.rpoplpush(bfoo, bdst));
+
+    assertArrayEquals(bC, belement);
+
+    List<byte[]> bsrcExpected = new ArrayList<>();
+    bsrcExpected.add(bA);
+    bsrcExpected.add(bB);
+
+    List<byte[]> bdstExpected = new ArrayList<>();
+    bdstExpected.add(bC);
+    bdstExpected.add(bfoo);
+    bdstExpected.add(bbar);
+
+    assertByteArrayListEquals(bsrcExpected, jedis.lrange(bfoo, 0, 1000));
+    assertByteArrayListEquals(bdstExpected, jedis.lrange(bdst, 0, 1000));
+  }
+
+  @Test
+  public void lindex() {
+    jedis.lpush("foo", "1");
+    jedis.lpush("foo", "2");
+    jedis.lpush("foo", "3");
+
+    assertEquals("3", blocking(batchJedis.lindex("foo", 0)));
+    assertEquals(null, blocking(batchJedis.lindex("foo", 100)));
+
+    // Binary
+    jedis.lpush(bfoo, b1);
+    jedis.lpush(bfoo, b2);
+    jedis.lpush(bfoo, b3);
+
+    assertArrayEquals(b3, blocking(batchJedis.lindex(bfoo, 0)));
+    assertEquals(null, blocking(batchJedis.lindex(bfoo, 100)));
+  }
+
+  @Test
+  public void linsert() {
+    long status = blocking(batchJedis.linsert("foo", LIST_POSITION.BEFORE, "bar", "car"));
+    assertEquals(0, status);
+
+    jedis.lpush("foo", "a");
+    status = blocking(batchJedis.linsert("foo", LIST_POSITION.AFTER, "a", "b"));
+    assertEquals(2, status);
+
+    List<String> actual = jedis.lrange("foo", 0, 100);
+    List<String> expected = new ArrayList<>();
+    expected.add("a");
+    expected.add("b");
+
+    assertEquals(expected, actual);
+
+    status = blocking(batchJedis.linsert("foo", LIST_POSITION.BEFORE, "bar", "car"));
+    assertEquals(-1, status);
+
+    // Binary
+    long bstatus = blocking(batchJedis.linsert(bfoo, LIST_POSITION.BEFORE, bbar, bcar));
+    assertEquals(0, bstatus);
+
+    jedis.lpush(bfoo, bA);
+    bstatus = blocking(batchJedis.linsert(bfoo, LIST_POSITION.AFTER, bA, bB));
+    assertEquals(2, bstatus);
+
+    List<byte[]> bactual = jedis.lrange(bfoo, 0, 100);
+    List<byte[]> bexpected = new ArrayList<>();
+    bexpected.add(bA);
+    bexpected.add(bB);
+
+    assertByteArrayListEquals(bexpected, bactual);
+
+    bstatus = blocking(batchJedis.linsert(bfoo, LIST_POSITION.BEFORE, bbar, bcar));
+    assertEquals(-1, bstatus);
+  }
+
+  @Test
+  public void llen() {
+    assertEquals(0, blocking(batchJedis.llen("foo")).intValue());
+    jedis.lpush("foo", "bar");
+    jedis.lpush("foo", "car");
+    assertEquals(2, blocking(batchJedis.llen("foo")).intValue());
+
+    // Binary
+    assertEquals(0, blocking(batchJedis.llen(bfoo)).intValue());
+    jedis.lpush(bfoo, bbar);
+    jedis.lpush(bfoo, bcar);
+    assertEquals(2, blocking(batchJedis.llen(bfoo)).intValue());
+  }
+
+  @Test
+  public void llenNotOnList() throws InterruptedException {
+    try {
+      jedis.set("foo", "bar");
+      MessageFuture<Long> fooFuture = batchJedis.llen("foo");
+      reporter.flush();
+      fooFuture.get();
+      fail("ExecutionException expected");
+    } catch (final ExecutionException e) {
+    }
+
+    // Binary
+    try {
+      jedis.set(bfoo, bbar);
+      MessageFuture<Long> bfooFuture = batchJedis.llen(bfoo);
+      reporter.flush();
+      bfooFuture.get();
+      fail("ExecutionException expected");
+    } catch (final ExecutionException e) {
+    }
+  }
+
+  @Test
+  public void lpop() {
+    jedis.rpush("foo", "a");
+    jedis.rpush("foo", "b");
+    jedis.rpush("foo", "c");
+
+    String element = blocking(batchJedis.lpop("foo"));
+    assertEquals("a", element);
+
+    List<String> expected = new ArrayList<>();
+    expected.add("b");
+    expected.add("c");
+
+    assertEquals(expected, jedis.lrange("foo", 0, 1000));
+    blocking(batchJedis.lpop("foo"));
+    blocking(batchJedis.lpop("foo"));
+
+    element = blocking(batchJedis.lpop("foo"));
+    assertNull(element);
+
+    // Binary
+    jedis.rpush(bfoo, bA);
+    jedis.rpush(bfoo, bB);
+    jedis.rpush(bfoo, bC);
+
+    byte[] belement = blocking(batchJedis.lpop(bfoo));
+    assertArrayEquals(bA, belement);
+
+    List<byte[]> bexpected = new ArrayList<>();
+    bexpected.add(bB);
+    bexpected.add(bC);
+
+    assertByteArrayListEquals(bexpected, jedis.lrange(bfoo, 0, 1000));
+    blocking(batchJedis.lpop(bfoo));
+    blocking(batchJedis.lpop(bfoo));
+
+    belement = blocking(batchJedis.lpop(bfoo));
+    assertNull(belement);
+  }
+}
diff --git a/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTest.java b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTest.java
new file mode 100644
index 0000000..b3d1f27
--- /dev/null
+++ b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTest.java
@@ -0,0 +1,791 @@
+package io.github.tramchamploo.bufferslayer;
+
+import static io.github.tramchamploo.bufferslayer.AssertUtil.arrayContains;
+import static io.github.tramchamploo.bufferslayer.AssertUtil.assertByteArrayListEquals;
+import static io.github.tramchamploo.bufferslayer.AssertUtil.assertByteArraySetEquals;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import io.github.tramchamploo.bufferslayer.internal.MessageFuture;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import redis.clients.jedis.BitPosParams;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.Pipeline;
+import redis.clients.jedis.Protocol;
+import redis.clients.jedis.exceptions.JedisDataException;
+import redis.clients.util.SafeEncoder;
+
+public class BatchJedisTest extends BatchJedisTestBase {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void pipeline() {
+    JedisPool jedisPool = mock(JedisPool.class);
+    reporter = AsyncReporter.builder(new JedisSender(jedisPool))
+        .messageTimeout(0, TimeUnit.MILLISECONDS)
+        .build();
+
+    batchJedis = new BatchJedis(jedisPool, reporter);
+
+    Jedis jedis = mock(Jedis.class);
+    when(jedisPool.getResource()).thenReturn(jedis);
+
+    Pipeline pipeline = mock(Pipeline.class);
+    when(pipeline.syncAndReturnAll()).thenReturn(Arrays.asList(1L, 1L));
+    when(jedis.pipelined()).thenReturn(pipeline);
+
+    batchJedis.append("foo", "bar");
+    batchJedis.append("foo", "bar");
+    reporter.flush();
+
+    verify(jedis).pipelined();
+    verify(pipeline, times(2))
+        .append(SafeEncoder.encode("foo"), SafeEncoder.encode("bar"));
+    verify(pipeline).syncAndReturnAll();
+  }
+
+  @Test
+  public void pipelineWithError() throws ExecutionException, InterruptedException {
+    jedis.set("foo", "bar");
+    MessageFuture<Long> barFuture = batchJedis.incr("bar");
+    MessageFuture<Long> fooFuture = batchJedis.incr("foo");
+    reporter.flush();
+    assertEquals(1, barFuture.get().intValue());
+    try {
+      fooFuture.get();
+    } catch (Throwable t) {
+      assertEquals(JedisDataException.class, t.getCause().getCause().getClass());
+    }
+  }
+
+  @Test
+  public void append() {
+    long value = blocking(batchJedis.append("foo", "bar"));
+    assertEquals(3L, value);
+    assertEquals("bar", jedis.get("foo"));
+
+    value = blocking(batchJedis.append("foo", "bar"));
+    assertEquals(6L, value);
+    assertEquals("barbar", jedis.get("foo"));
+  }
+
+  @Test
+  public void blpop() {
+    List<String> value = blocking(batchJedis.blpop(1, "foo"));
+    assertNull(value);
+
+    jedis.lpush("foo", "bar");
+    value = blocking(batchJedis.blpop(1, "foo"));
+    assertEquals(2, value.size());
+    assertEquals("foo", value.get(0));
+    assertEquals("bar", value.get(1));
+
+    // Binary
+    jedis.lpush(bfoo, bbar);
+    List<byte[]> value2 = blocking(batchJedis.blpop(1, bfoo));
+    assertEquals(2, value2.size());
+    assertArrayEquals(bfoo, value2.get(0));
+    assertArrayEquals(bbar, value2.get(1));
+  }
+
+  @Test
+  public void brpop() {
+    List<String> value = blocking(batchJedis.brpop(1, "foo"));
+    assertNull(value);
+
+    jedis.lpush("foo", "bar");
+    value = blocking(batchJedis.brpop(1, "foo"));
+    assertEquals(2, value.size());
+    assertEquals("foo", value.get(0));
+    assertEquals("bar", value.get(1));
+
+    // Binary
+    jedis.lpush(bfoo, bbar);
+    List<byte[]> value2 = blocking(batchJedis.brpop(1, bfoo));
+    assertEquals(2, value2.size());
+    assertArrayEquals(bfoo, value2.get(0));
+    assertArrayEquals(bbar, value2.get(1));
+  }
+
+  @Test
+  public void decr() {
+    assertEquals(new Long(1L), jedis.incr("foo"));
+    assertEquals(new Long(0L), blocking(batchJedis.decr("foo")));
+
+    // Binary
+    assertEquals(new Long(1L), jedis.incr(bfoo));
+    assertEquals(new Long(0L), blocking(batchJedis.decr(bfoo)));
+  }
+
+  @Test
+  public void decrBy() {
+    assertEquals(new Long(2L), jedis.incrBy("foo", 2));
+    assertEquals(new Long(0L), blocking(batchJedis.decrBy("foo", 2)));
+
+    // Binary
+    assertEquals(new Long(2L), jedis.incrBy(bfoo, 2));
+    assertEquals(new Long(0L), blocking(batchJedis.decrBy(bfoo, 2)));
+  }
+
+  @Test
+  public void del() {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals(new Long(1L), blocking(batchJedis.del("foo")));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertEquals(new Long(1L), blocking(batchJedis.del(bfoo)));
+  }
+
+  @Test
+  public void echo() {
+    assertEquals("foo", blocking(batchJedis.echo("foo")));
+
+    // Binary
+    assertArrayEquals(bfoo, blocking(batchJedis.echo(bfoo)));
+  }
+
+  @Test
+  public void exists() {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals(true, blocking(batchJedis.exists("foo")));
+
+    assertEquals("OK", jedis.set("bar", "bar"));
+    assertEquals(new Long(2L), blocking(batchJedis.exists("foo", "bar")));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertEquals(true, blocking(batchJedis.exists(bfoo)));
+
+    assertEquals("OK", jedis.set(bbar, bbar));
+    assertEquals(new Long(2L), blocking(batchJedis.exists(bfoo, bbar)));
+  }
+
+  @Test
+  public void expire() throws InterruptedException {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals(new Long(1L), blocking(batchJedis.expire("foo", 1)));
+    Thread.sleep(1200);
+    assertFalse(jedis.exists("foo"));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertEquals(new Long(1L), blocking(batchJedis.expire(bfoo, 1)));
+    Thread.sleep(1200);
+    assertFalse(jedis.exists(bfoo));
+  }
+
+  @Test
+  public void expireAt() throws InterruptedException {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals(new Long(1L), blocking(
+        batchJedis.expireAt("foo", System.currentTimeMillis() / 1000 + 1)));
+    Thread.sleep(1000);
+    assertFalse(jedis.exists("foo"));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertEquals(new Long(1L), blocking(
+        batchJedis.expireAt(bfoo, System.currentTimeMillis() / 1000 + 1)));
+    Thread.sleep(1000);
+    assertFalse(jedis.exists(bfoo));
+  }
+
+  @Test
+  public void get() {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals("bar", blocking(batchJedis.get("foo")));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertArrayEquals(bbar, blocking(batchJedis.get(bfoo)));
+  }
+
+  @Test
+  public void getbit() {
+    boolean bit = jedis.setbit("foo", 0, true);
+    assertEquals(false, bit);
+
+    bit = blocking(batchJedis.getbit("foo", 0));
+    assertEquals(true, bit);
+
+    boolean bbit = jedis.setbit("bfoo".getBytes(), 0, "1".getBytes());
+    assertFalse(bbit);
+
+    bbit = blocking(batchJedis.getbit("bfoo".getBytes(), 0));
+    assertTrue(bbit);
+  }
+
+  @Test
+  public void bitpos() {
+    String foo = "foo";
+
+    jedis.set(foo, String.valueOf(0));
+
+    jedis.setbit(foo, 3, true);
+    jedis.setbit(foo, 7, true);
+    jedis.setbit(foo, 13, true);
+    jedis.setbit(foo, 39, true);
+
+    /*
+     * byte: 0 1 2 3 4 bit: 00010001 / 00000100 / 00000000 / 00000000 / 00000001
+     */
+    long offset = blocking(batchJedis.bitpos(foo, true));
+    assertEquals(2, offset);
+    offset = blocking(batchJedis.bitpos(foo, false));
+    assertEquals(0, offset);
+
+    offset = blocking(batchJedis.bitpos(foo, true, new redis.clients.jedis.BitPosParams(1)));
+    assertEquals(13, offset);
+    offset = blocking(batchJedis.bitpos(foo, false, new redis.clients.jedis.BitPosParams(1)));
+    assertEquals(8, offset);
+
+    offset = blocking(batchJedis.bitpos(foo, true, new redis.clients.jedis.BitPosParams(2, 3)));
+    assertEquals(-1, offset);
+    offset = blocking(batchJedis.bitpos(foo, false, new redis.clients.jedis.BitPosParams(2, 3)));
+    assertEquals(16, offset);
+
+    offset = blocking(batchJedis.bitpos(foo, true, new redis.clients.jedis.BitPosParams(3, 4)));
+    assertEquals(39, offset);
+  }
+
+  @Test
+  public void bitposBinary() {
+    // binary
+    byte[] bfoo = { 0x01, 0x02, 0x03, 0x04 };
+
+    jedis.set(bfoo, Protocol.toByteArray(0));
+
+    jedis.setbit(bfoo, 3, true);
+    jedis.setbit(bfoo, 7, true);
+    jedis.setbit(bfoo, 13, true);
+    jedis.setbit(bfoo, 39, true);
+
+    /*
+     * byte: 0 1 2 3 4 bit: 00010001 / 00000100 / 00000000 / 00000000 / 00000001
+     */
+    long offset = blocking(batchJedis.bitpos(bfoo, true));
+    assertEquals(2, offset);
+    offset = blocking(batchJedis.bitpos(bfoo, false));
+    assertEquals(0, offset);
+
+    offset = blocking(batchJedis.bitpos(bfoo, true, new redis.clients.jedis.BitPosParams(1)));
+    assertEquals(13, offset);
+    offset = blocking(batchJedis.bitpos(bfoo, false, new redis.clients.jedis.BitPosParams(1)));
+    assertEquals(8, offset);
+
+    offset = blocking(batchJedis.bitpos(bfoo, true, new redis.clients.jedis.BitPosParams(2, 3)));
+    assertEquals(-1, offset);
+    offset = blocking(batchJedis.bitpos(bfoo, false, new redis.clients.jedis.BitPosParams(2, 3)));
+    assertEquals(16, offset);
+
+    offset = blocking(batchJedis.bitpos(bfoo, true, new redis.clients.jedis.BitPosParams(3, 4)));
+    assertEquals(39, offset);
+  }
+
+  @Test
+  public void bitposWithNoMatchingBitExist() {
+    String foo = "foo";
+
+    jedis.set(foo, String.valueOf(0));
+    for (int idx = 0; idx < 8; idx++) {
+      jedis.setbit(foo, idx, true);
+    }
+
+    /*
+     * byte: 0 bit: 11111111
+     */
+    long offset = blocking(batchJedis.bitpos(foo, false));
+    // offset should be last index + 1
+    assertEquals(8, offset);
+  }
+
+  @Test
+  public void bitposWithNoMatchingBitExistWithinRange() {
+    String foo = "foo";
+
+    jedis.set(foo, String.valueOf(0));
+    for (int idx = 0; idx < 8 * 5; idx++) {
+      jedis.setbit(foo, idx, true);
+    }
+
+    /*
+     * byte: 0 1 2 3 4 bit: 11111111 / 11111111 / 11111111 / 11111111 / 11111111
+     */
+    long offset = blocking(batchJedis.bitpos(foo, false, new BitPosParams(2, 3)));
+    // offset should be -1
+    assertEquals(-1, offset);
+  }
+
+  @Test
+  public void getRange() {
+    assertEquals("OK", jedis.set("foo", "bar"));
+    assertEquals("a", blocking(batchJedis.getrange("foo", 1, 1)));
+
+    // Binary
+    assertEquals("OK", jedis.set(bfoo, bbar));
+    assertArrayEquals(new byte[]{0x06}, blocking(batchJedis.getrange(bfoo, 1, 1)));
+  }
+
+  @Test
+  public void setAndGet() {
+    String status = blocking(batchJedis.set("foo", "bar"));
+    assertEquals("OK", status);
+
+    String value = jedis.get("foo");
+    assertEquals("bar", value);
+
+    // Binary
+    status = blocking(batchJedis.set(bfoo, bbar));
+    assertEquals("OK", status);
+    byte[] bvalue = jedis.get(bfoo);
+    assertArrayEquals(bbar, bvalue);
+  }
+
+  @Test
+  public void getSet() {
+    String status = jedis.set("foo", "bar");
+    assertEquals("OK", status);
+
+    String value = blocking(batchJedis.getSet("foo", "foo"));
+    assertEquals("bar", value);
+    assertEquals("foo", jedis.get("foo"));
+
+    // Binary
+    status = jedis.set(bfoo, bbar);
+    assertEquals("OK", status);
+    byte[] bvalue = blocking(batchJedis.getSet(bfoo, bfoo));
+    assertArrayEquals(bbar, bvalue);
+    assertArrayEquals(bfoo, jedis.get(bfoo));
+  }
+
+  @Test
+  public void hdel() {
+    Map<String, String> hash = new HashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    assertEquals(0, blocking(batchJedis.hdel("bar", "foo")).intValue());
+    assertEquals(0, blocking(batchJedis.hdel("foo", "foo")).intValue());
+    assertEquals(1, blocking(batchJedis.hdel("foo", "bar")).intValue());
+    assertEquals(null, jedis.hget("foo", "bar"));
+
+    // Binary
+    Map<byte[], byte[]> bhash = new HashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    assertEquals(0, blocking(batchJedis.hdel(bbar, bfoo)).intValue());
+    assertEquals(0, blocking(batchJedis.hdel(bfoo, bfoo)).intValue());
+    assertEquals(1, blocking(batchJedis.hdel(bfoo, bbar)).intValue());
+    assertEquals(null, jedis.hget(bfoo, bbar));
+  }
+
+  @Test
+  public void hexists() {
+    Map<String, String> hash = new HashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    assertFalse(blocking(batchJedis.hexists("bar", "foo")));
+    assertFalse(blocking(batchJedis.hexists("foo", "foo")));
+    assertTrue(blocking(batchJedis.hexists("foo", "bar")));
+
+    // Binary
+    Map<byte[], byte[]> bhash = new HashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    assertFalse(blocking(batchJedis.hexists(bbar, bfoo)));
+    assertFalse(blocking(batchJedis.hexists(bfoo, bfoo)));
+    assertTrue(blocking(batchJedis.hexists(bfoo, bbar)));
+  }
+
+  @Test
+  public void hget() {
+    jedis.hset("foo", "bar", "car");
+    assertEquals(null, blocking(batchJedis.hget("bar", "foo")));
+    assertEquals(null, blocking(batchJedis.hget("foo", "car")));
+    assertEquals("car", blocking(batchJedis.hget("foo", "bar")));
+
+    // Binary
+    jedis.hset(bfoo, bbar, bcar);
+    assertEquals(null, blocking(batchJedis.hget(bbar, bfoo)));
+    assertEquals(null, blocking(batchJedis.hget(bfoo, bcar)));
+    assertArrayEquals(bcar, blocking(batchJedis.hget(bfoo, bbar)));
+  }
+
+  @Test
+  public void hgetAll() {
+    Map<String, String> h = new HashMap<>();
+    h.put("bar", "car");
+    h.put("car", "bar");
+    jedis.hmset("foo", h);
+
+    Map<String, String> hash = blocking(batchJedis.hgetAll("foo"));
+    assertEquals(2, hash.size());
+    assertEquals("car", hash.get("bar"));
+    assertEquals("bar", hash.get("car"));
+
+    // Binary
+    Map<byte[], byte[]> bh = new HashMap<>();
+    bh.put(bbar, bcar);
+    bh.put(bcar, bbar);
+    jedis.hmset(bfoo, bh);
+    Map<byte[], byte[]> bhash = blocking(batchJedis.hgetAll(bfoo));
+
+    assertEquals(2, bhash.size());
+    assertArrayEquals(bcar, bhash.get(bbar));
+    assertArrayEquals(bbar, bhash.get(bcar));
+  }
+
+  @Test
+  public void hincrBy() {
+    long value = blocking(batchJedis.hincrBy("foo", "bar", 1));
+    assertEquals(1, value);
+    value = blocking(batchJedis.hincrBy("foo", "bar", -1));
+    assertEquals(0, value);
+    value = blocking(batchJedis.hincrBy("foo", "bar", -10));
+    assertEquals(-10, value);
+
+    // Binary
+    long bvalue = blocking(batchJedis.hincrBy(bfoo, bbar, 1));
+    assertEquals(1, bvalue);
+    bvalue = blocking(batchJedis.hincrBy(bfoo, bbar, -1));
+    assertEquals(0, bvalue);
+    bvalue = blocking(batchJedis.hincrBy(bfoo, bbar, -10));
+    assertEquals(-10, bvalue);
+  }
+
+  @Test
+  public void hkeys() {
+    Map<String, String> hash = new LinkedHashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    Set<String> keys = blocking(batchJedis.hkeys("foo"));
+    Set<String> expected = new LinkedHashSet<>();
+    expected.add("bar");
+    expected.add("car");
+    assertEquals(expected, keys);
+
+    // Binary
+    Map<byte[], byte[]> bhash = new LinkedHashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    Set<byte[]> bkeys = blocking(batchJedis.hkeys(bfoo));
+    Set<byte[]> bexpected = new LinkedHashSet<>();
+    bexpected.add(bbar);
+    bexpected.add(bcar);
+    assertByteArraySetEquals(bexpected, bkeys);
+  }
+
+  @Test
+  public void hlen() {
+    Map<String, String> hash = new HashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    assertEquals(0, blocking(batchJedis.hlen("bar")).intValue());
+    assertEquals(2, blocking(batchJedis.hlen("foo")).intValue());
+
+    // Binary
+    Map<byte[], byte[]> bhash = new HashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    assertEquals(0, blocking(batchJedis.hlen(bbar)).intValue());
+    assertEquals(2, blocking(batchJedis.hlen(bfoo)).intValue());
+  }
+
+  @Test
+  public void hmget() {
+    Map<String, String> hash = new HashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    List<String> values = blocking(batchJedis.hmget("foo", "bar", "car", "foo"));
+    List<String> expected = new ArrayList<>();
+    expected.add("car");
+    expected.add("bar");
+    expected.add(null);
+
+    assertEquals(expected, values);
+
+    // Binary
+    Map<byte[], byte[]> bhash = new HashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    List<byte[]> bvalues = blocking(batchJedis.hmget(bfoo, bbar, bcar, bfoo));
+    List<byte[]> bexpected = new ArrayList<>();
+    bexpected.add(bcar);
+    bexpected.add(bbar);
+    bexpected.add(null);
+
+    assertByteArrayListEquals(bexpected, bvalues);
+  }
+
+  @Test
+  public void hmset() {
+    Map<String, String> hash = new HashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    String status = blocking(batchJedis.hmset("foo", hash));
+    assertEquals("OK", status);
+    assertEquals("car", jedis.hget("foo", "bar"));
+    assertEquals("bar", jedis.hget("foo", "car"));
+
+    // Binary
+    Map<byte[], byte[]> bhash = new HashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    String bstatus = blocking(batchJedis.hmset(bfoo, bhash));
+    assertEquals("OK", bstatus);
+    assertArrayEquals(bcar, jedis.hget(bfoo, bbar));
+    assertArrayEquals(bbar, jedis.hget(bfoo, bcar));
+  }
+
+  @Test
+  public void hset() {
+    long status = blocking(batchJedis.hset("foo", "bar", "car"));
+    assertEquals(1, status);
+    status = blocking(batchJedis.hset("foo", "bar", "foo"));
+    assertEquals(0, status);
+
+    // Binary
+    long bstatus = blocking(batchJedis.hset(bfoo, bbar, bcar));
+    assertEquals(1, bstatus);
+    bstatus = blocking(batchJedis.hset(bfoo, bbar, bfoo));
+    assertEquals(0, bstatus);
+  }
+
+  @Test
+  public void hsetnx() {
+    long status = blocking(batchJedis.hsetnx("foo", "bar", "car"));
+    assertEquals(1, status);
+    assertEquals("car", jedis.hget("foo", "bar"));
+
+    status = blocking(batchJedis.hsetnx("foo", "bar", "foo"));
+    assertEquals(0, status);
+    assertEquals("car", jedis.hget("foo", "bar"));
+
+    status = blocking(batchJedis.hsetnx("foo", "car", "bar"));
+    assertEquals(1, status);
+    assertEquals("bar", jedis.hget("foo", "car"));
+
+    // Binary
+    long bstatus = blocking(batchJedis.hsetnx(bfoo, bbar, bcar));
+    assertEquals(1, bstatus);
+    assertArrayEquals(bcar, jedis.hget(bfoo, bbar));
+
+    bstatus = blocking(batchJedis.hsetnx(bfoo, bbar, bfoo));
+    assertEquals(0, bstatus);
+    assertArrayEquals(bcar, jedis.hget(bfoo, bbar));
+
+    bstatus = blocking(batchJedis.hsetnx(bfoo, bcar, bbar));
+    assertEquals(1, bstatus);
+    assertArrayEquals(bbar, jedis.hget(bfoo, bcar));
+  }
+
+  @Test
+  public void hvals() {
+    Map<String, String> hash = new LinkedHashMap<>();
+    hash.put("bar", "car");
+    hash.put("car", "bar");
+    jedis.hmset("foo", hash);
+
+    List<String> vals = blocking(batchJedis.hvals("foo"));
+    assertEquals(2, vals.size());
+    assertTrue(vals.contains("bar"));
+    assertTrue(vals.contains("car"));
+
+    // Binary
+    Map<byte[], byte[]> bhash = new LinkedHashMap<>();
+    bhash.put(bbar, bcar);
+    bhash.put(bcar, bbar);
+    jedis.hmset(bfoo, bhash);
+
+    List<byte[]> bvals = blocking(batchJedis.hvals(bfoo));
+
+    assertEquals(2, bvals.size());
+    assertTrue(arrayContains(bvals, bbar));
+    assertTrue(arrayContains(bvals, bcar));
+  }
+
+  @Test
+  public void incrWrongValue() {
+    jedis.set("foo", "bar");
+    try {
+      MessageFuture<Long> future = batchJedis.incr("foo");
+      reporter.flush();
+      future.get();
+    } catch (Throwable t) {
+      assertEquals(JedisDataException.class, t.getCause().getCause().getClass());
+    }
+  }
+
+  @Test
+  public void incr() {
+    long value = blocking(batchJedis.incr("foo"));
+    assertEquals(1, value);
+    value = blocking(batchJedis.incr("foo"));
+    assertEquals(2, value);
+  }
+
+  @Test
+  public void incrByWrongValue() {
+    jedis.set("foo", "bar");
+    try {
+      MessageFuture<Long> future = batchJedis.incrBy("foo", 2);
+      reporter.flush();
+      future.get();
+    } catch (Throwable t) {
+      assertEquals(JedisDataException.class, t.getCause().getCause().getClass());
+    }
+  }
+
+  @Test
+  public void incrBy() {
+    long value = blocking(batchJedis.incrBy("foo", 2));
+    assertEquals(2, value);
+    value = blocking(batchJedis.incrBy("foo", 2));
+    assertEquals(4, value);
+  }
+
+  @Test
+  public void mget() {
+    List<String> values = blocking(batchJedis.mget("foo", "bar"));
+    List<String> expected = new ArrayList<>();
+    expected.add(null);
+    expected.add(null);
+
+    assertEquals(expected, values);
+
+    jedis.set("foo", "bar");
+
+    expected = new ArrayList<>();
+    expected.add("bar");
+    expected.add(null);
+    values = blocking(batchJedis.mget("foo", "bar"));
+
+    assertEquals(expected, values);
+
+    jedis.set("bar", "foo");
+
+    expected = new ArrayList<>();
+    expected.add("bar");
+    expected.add("foo");
+    values = blocking(batchJedis.mget("foo", "bar"));
+
+    assertEquals(expected, values);
+  }
+
+  @Test
+  public void mset() {
+    String status = blocking(batchJedis.mset("foo", "bar", "bar", "foo"));
+    assertEquals("OK", status);
+    assertEquals("bar", jedis.get("foo"));
+    assertEquals("foo", jedis.get("bar"));
+  }
+
+  @Test
+  public void msetnx() {
+    long status = blocking(batchJedis.msetnx("foo", "bar", "bar", "foo"));
+    assertEquals(1, status);
+    assertEquals("bar", jedis.get("foo"));
+    assertEquals("foo", jedis.get("bar"));
+
+    status = blocking(batchJedis.msetnx("foo", "bar1", "bar2", "foo2"));
+    assertEquals(0, status);
+    assertEquals("bar", jedis.get("foo"));
+    assertEquals("foo", jedis.get("bar"));
+  }
+
+  @Test
+  public void rename() {
+    jedis.set("foo", "bar");
+    String status = blocking(batchJedis.rename("foo", "bar"));
+    assertEquals("OK", status);
+
+    String value = jedis.get("foo");
+    assertEquals(null, value);
+
+    value = jedis.get("bar");
+    assertEquals("bar", value);
+
+    // Binary
+    jedis.set(bfoo, bbar);
+    String bstatus = blocking(batchJedis.rename(bfoo, bbar));
+    assertEquals("OK", bstatus);
+
+    byte[] bvalue = jedis.get(bfoo);
+    assertEquals(null, bvalue);
+
+    bvalue = jedis.get(bbar);
+    assertArrayEquals(bbar, bvalue);
+  }
+
+  @Test
+  public void renameOldAndNewAreTheSame() {
+    jedis.set("foo", "bar");
+    blocking(batchJedis.rename("foo", "foo"));
+
+    // Binary
+    jedis.set(bfoo, bbar);
+    blocking(batchJedis.rename(bfoo, bfoo));
+  }
+
+  @Test
+  public void renamenx() {
+    jedis.set("foo", "bar");
+    long status = blocking(batchJedis.renamenx("foo", "bar"));
+    assertEquals(1, status);
+
+    jedis.set("foo", "bar");
+    status = blocking(batchJedis.renamenx("foo", "bar"));
+    assertEquals(0, status);
+
+    // Binary
+    jedis.set(bfoo, bbar);
+    long bstatus = blocking(batchJedis.renamenx(bfoo, bbar));
+    assertEquals(1, bstatus);
+
+    jedis.set(bfoo, bbar);
+    bstatus = blocking(batchJedis.renamenx(bfoo, bbar));
+    assertEquals(0, bstatus);
+  }
+}
diff --git a/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTestBase.java b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTestBase.java
new file mode 100644
index 0000000..7f02248
--- /dev/null
+++ b/jedis/src/test/java/io/github/tramchamploo/bufferslayer/BatchJedisTestBase.java
@@ -0,0 +1,43 @@
+package io.github.tramchamploo.bufferslayer;
+
+import io.github.tramchamploo.bufferslayer.internal.MessageFuture;
+import java.util.concurrent.TimeUnit;
+import junit.framework.AssertionFailedError;
+import org.junit.Before;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisPool;
+
+public abstract class BatchJedisTestBase {
+
+  protected final byte[] bfoo = {0x01, 0x02, 0x03, 0x04};
+  protected final byte[] bbar = {0x05, 0x06, 0x07, 0x08};
+  protected final byte[] bcar = { 0x09, 0x0A, 0x0B, 0x0C };
+
+  protected Jedis jedis;
+  protected BatchJedis batchJedis;
+  protected AsyncReporter reporter;
+  protected JedisPool jedisPool;
+
+  @Before
+  public void setup() {
+    jedis = new Jedis("localhost", 6379);
+    jedis.flushAll();
+
+    jedisPool = new JedisPool();
+    reporter = AsyncReporter.builder(new JedisSender(jedisPool))
+        .messageTimeout(0, TimeUnit.MILLISECONDS)
+        .build();
+    batchJedis = new BatchJedis(jedisPool, reporter);
+  }
+
+  @SuppressWarnings("unchecked")
+  <T> T blocking(MessageFuture<T> future) {
+    reporter.flush();
+
+    try {
+      return future.get();
+    } catch (Exception e) {
+      throw new AssertionFailedError(e.getMessage());
+    }
+  }
+}
diff --git a/pom.xml b/pom.xml
index 1381f92..9f94103 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,6 +22,7 @@
     <module>jdbc</module>
     <module>benchmark</module>
     <module>rx</module>
+    <module>jedis</module>
   </modules>
 
   <properties>
@@ -109,6 +110,12 @@
         <type>test-jar</type>
       </dependency>
 
+      <dependency>
+        <groupId>${project.groupId}</groupId>
+        <artifactId>bufferslayer-jedis</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>${project.groupId}</groupId>
         <artifactId>bufferslayer-rx</artifactId>
@@ -168,6 +175,12 @@
         <artifactId>h2</artifactId>
         <version>1.4.196</version>
       </dependency>
+
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-core</artifactId>
+        <version>2.7.22</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
@@ -237,6 +250,16 @@
     </pluginManagement>
 
     <plugins>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>${maven.compiler.source}</source>
+          <target>${maven.compiler.target}</target>
+          <testSource>${maven.compiler.testSource}</testSource>
+          <testTarget>${maven.compiler.testTarget}</testTarget>
+        </configuration>
+      </plugin>
+
       <plugin>
         <artifactId>maven-release-plugin</artifactId>
         <version>${maven-release-plugin.version}</version>
@@ -331,4 +354,4 @@
       </properties>
     </profile>
   </profiles>
-</project>
\ No newline at end of file
+</project>