You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by am...@apache.org on 2014/11/13 12:50:00 UTC

[33/50] incubator-lens git commit: Moving Mock classes to a more convenient/accessible location: inside lens-server-api

Moving Mock classes to a more convenient/accessible location: inside lens-server-api


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/180f6b41
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/180f6b41
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/180f6b41

Branch: refs/heads/toapache
Commit: 180f6b4146b2dd17cae02df3ae434dcd52154d90
Parents: 90dfa2a
Author: Rajat Khandelwal <ra...@inmobi.com>
Authored: Wed Nov 12 16:02:55 2014 +0530
Committer: Rajat Khandelwal <ra...@inmobi.com>
Committed: Wed Nov 12 16:02:55 2014 +0530

----------------------------------------------------------------------
 .../org/apache/lens/driver/cube/MockDriver.java | 345 ------------------
 .../lens/driver/cube/TestMinCostSelector.java   |  33 +-
 .../apache/lens/driver/cube/TestRewriting.java  |   3 +-
 lens-cube/src/test/resources/hive-site.xml      |   2 +-
 .../server/api/driver/MinQueryCostSelector.java |  53 +--
 .../lens/server/api/driver/MockDriver.java      | 361 +++++++++++++++++++
 .../lens/server/api/driver/MockFailDriver.java  |  37 ++
 .../server/api/driver/MockQueryContext.java     |  30 ++
 8 files changed, 453 insertions(+), 411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-cube/src/test/java/org/apache/lens/driver/cube/MockDriver.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/driver/cube/MockDriver.java b/lens-cube/src/test/java/org/apache/lens/driver/cube/MockDriver.java
deleted file mode 100644
index 5b470de..0000000
--- a/lens-cube/src/test/java/org/apache/lens/driver/cube/MockDriver.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.lens.driver.cube;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hive.service.cli.ColumnDescriptor;
-import org.apache.lens.api.LensException;
-import org.apache.lens.api.query.QueryCost;
-import org.apache.lens.api.query.QueryHandle;
-import org.apache.lens.api.query.QueryPrepareHandle;
-import org.apache.lens.api.query.ResultRow;
-import org.apache.lens.server.api.driver.*;
-import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
-import org.apache.lens.server.api.events.LensEventListener;
-import org.apache.lens.server.api.query.PreparedQueryContext;
-import org.apache.lens.server.api.query.QueryContext;
-
-/**
- * The Class MockDriver.
- */
-public class MockDriver implements LensDriver {
-
-  /** The conf. */
-  Configuration conf;
-
-  /** The query. */
-  String query;
-
-  /** The io test val. */
-  private int ioTestVal = -1;
-
-  /**
-   * Instantiates a new mock driver.
-   */
-  public MockDriver() {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#configure(org.apache.hadoop.conf.Configuration)
-   */
-  @Override
-  public void configure(Configuration conf) throws LensException {
-    this.conf = conf;
-    ioTestVal = conf.getInt("mock.driver.test.val", -1);
-  }
-
-  /**
-   * The Class MockQueryPlan.
-   */
-  static class MockQueryPlan extends DriverQueryPlan {
-
-    /** The query. */
-    String query;
-
-    /**
-     * Instantiates a new mock query plan.
-     *
-     * @param query
-     *          the query
-     */
-    MockQueryPlan(String query) {
-      this.query = query;
-      setPrepareHandle(new QueryPrepareHandle(UUID.randomUUID()));
-    }
-
-    @Override
-    public String getPlan() {
-      return query;
-    }
-
-    @Override
-    public QueryCost getCost() {
-      return new QueryCost(0L, 0.0);
-    }
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#explain(java.lang.String, org.apache.hadoop.conf.Configuration)
-   */
-  @Override
-  public DriverQueryPlan explain(String query, Configuration conf) throws LensException {
-    return new MockQueryPlan(query);
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#updateStatus(org.apache.lens.server.api.query.QueryContext)
-   */
-  @Override
-  public void updateStatus(QueryContext context) throws LensException {
-    context.getDriverStatus().setProgress(1.0);
-    context.getDriverStatus().setStatusMessage("Done");
-    context.getDriverStatus().setState(DriverQueryState.SUCCESSFUL);
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#cancelQuery(org.apache.lens.api.query.QueryHandle)
-   */
-  @Override
-  public boolean cancelQuery(QueryHandle handle) throws LensException {
-    return false;
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#closeQuery(org.apache.lens.api.query.QueryHandle)
-   */
-  @Override
-  public void closeQuery(QueryHandle handle) throws LensException {
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#close()
-   */
-  @Override
-  public void close() throws LensException {
-  }
-
-  /**
-   * Add a listener for driver events.
-   *
-   * @param driverEventListener
-   *          the driver event listener
-   */
-  @Override
-  public void registerDriverEventListener(LensEventListener<DriverEvent> driverEventListener) {
-
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#prepare(org.apache.lens.server.api.query.PreparedQueryContext)
-   */
-  @Override
-  public void prepare(PreparedQueryContext pContext) throws LensException {
-    // TODO Auto-generated method stub
-
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.lens.server.api.driver.LensDriver#explainAndPrepare(org.apache.lens.server.api.query.PreparedQueryContext
-   * )
-   */
-  @Override
-  public DriverQueryPlan explainAndPrepare(PreparedQueryContext pContext) throws LensException {
-    DriverQueryPlan p = new MockQueryPlan(pContext.getDriverQuery());
-    p.setPrepareHandle(pContext.getPrepareHandle());
-    return p;
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#closePreparedQuery(org.apache.lens.api.query.QueryPrepareHandle)
-   */
-  @Override
-  public void closePreparedQuery(QueryPrepareHandle handle) throws LensException {
-    // TODO Auto-generated method stub
-
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#execute(org.apache.lens.server.api.query.QueryContext)
-   */
-  @Override
-  public LensResultSet execute(QueryContext context) throws LensException {
-    this.query = context.getDriverQuery();
-    return new PersistentResultSet() {
-
-      @Override
-      public int size() throws LensException {
-        // TODO Auto-generated method stub
-        return 0;
-      }
-
-      @Override
-      public LensResultSetMetadata getMetadata() throws LensException {
-        // TODO Auto-generated method stub
-        return new LensResultSetMetadata() {
-
-          @Override
-          public List<ColumnDescriptor> getColumns() {
-            // TODO Auto-generated method stub
-            return null;
-          }
-        };
-      }
-
-      @Override
-      public String getOutputPath() throws LensException {
-        // TODO Auto-generated method stub
-        return null;
-      }
-    };
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#executeAsync(org.apache.lens.server.api.query.QueryContext)
-   */
-  @Override
-  public void executeAsync(QueryContext context) throws LensException {
-    this.query = context.getDriverQuery();
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#fetchResultSet(org.apache.lens.server.api.query.QueryContext)
-   */
-  @Override
-  public LensResultSet fetchResultSet(QueryContext context) throws LensException {
-    return new InMemoryResultSet() {
-
-      @Override
-      public int size() throws LensException {
-        // TODO Auto-generated method stub
-        return 0;
-      }
-
-      @Override
-      public LensResultSetMetadata getMetadata() throws LensException {
-        return new LensResultSetMetadata() {
-
-          @Override
-          public List<ColumnDescriptor> getColumns() {
-            // TODO Auto-generated method stub
-            return null;
-          }
-        };
-      }
-
-      @Override
-      public void setFetchSize(int size) throws LensException {
-        // TODO Auto-generated method stub
-
-      }
-
-      @Override
-      public ResultRow next() throws LensException {
-        // TODO Auto-generated method stub
-        return null;
-      }
-
-      @Override
-      public boolean hasNext() throws LensException {
-        // TODO Auto-generated method stub
-        return false;
-      }
-    };
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.lens.server.api.driver.LensDriver#closeResultSet(org.apache.lens.api.query.QueryHandle)
-   */
-  @Override
-  public void closeResultSet(QueryHandle handle) throws LensException {
-    // TODO Auto-generated method stub
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.lens.server.api.driver.LensDriver#registerForCompletionNotification(org.apache.lens.api.query.QueryHandle
-   * , long, org.apache.lens.server.api.driver.QueryCompletionListener)
-   */
-  @Override
-  public void registerForCompletionNotification(QueryHandle handle, long timeoutMillis, QueryCompletionListener listener)
-      throws LensException {
-    // TODO Auto-generated method stub
-
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see java.io.Externalizable#readExternal(java.io.ObjectInput)
-   */
-  @Override
-  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-    ioTestVal = in.readInt();
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see java.io.Externalizable#writeExternal(java.io.ObjectOutput)
-   */
-  @Override
-  public void writeExternal(ObjectOutput out) throws IOException {
-    out.writeInt(ioTestVal);
-  }
-
-  public int getTestIOVal() {
-    return ioTestVal;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
index 66de80c..a3df316 100644
--- a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
+++ b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestMinCostSelector.java
@@ -25,9 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.lens.api.LensException;
-import org.apache.lens.server.api.driver.DriverQueryPlan;
-import org.apache.lens.server.api.driver.LensDriver;
-import org.apache.lens.server.api.driver.MinQueryCostSelector;
+import org.apache.lens.server.api.driver.*;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -35,27 +33,8 @@ import org.testng.annotations.Test;
  * The Class TestMinCostSelector.
  */
 public class TestMinCostSelector {
-
-  /**
-   * The Class MockFailDriver.
-   */
-  static class MockFailDriver extends MockDriver {
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.lens.driver.cube.MockDriver#explain(java.lang.String, org.apache.hadoop.conf.Configuration)
-     */
-    public DriverQueryPlan explain(String query, Configuration conf) throws LensException {
-      throw new LensException("failing!");
-    }
-  }
-
-  /**
-   * Test min cost selector.
-   */
   @Test
-  public void testMinCostSelector() {
+  public void testMinCostSelector() throws LensException {
     MinQueryCostSelector selector = new MinQueryCostSelector();
     List<LensDriver> drivers = new ArrayList<LensDriver>();
     Map<LensDriver, String> driverQueries = new HashMap<LensDriver, String>();
@@ -69,21 +48,21 @@ public class TestMinCostSelector {
     drivers.add(d1);
     drivers.add(d2);
     driverQueries.put(d1, "test query");
-    LensDriver selected = selector.select(drivers, driverQueries, conf);
+    LensDriver selected = selector.select(new MockQueryContext(driverQueries), conf);
     Assert.assertEquals(d1, selected);
     driverQueries.put(d2, "test query");
     driverQueries.remove(d1);
-    selected = selector.select(drivers, driverQueries, conf);
+    selected = selector.select(new MockQueryContext(driverQueries), conf);
     Assert.assertEquals(d2, selected);
 
     drivers.add(fd1);
     driverQueries.put(fd1, "test query");
-    selected = selector.select(drivers, driverQueries, conf);
+    selected = selector.select(new MockQueryContext(driverQueries), conf);
     Assert.assertEquals(d2, selected);
 
     drivers.add(fd2);
     driverQueries.put(fd2, "test query");
-    selected = selector.select(drivers, driverQueries, conf);
+    selected = selector.select(new MockQueryContext(driverQueries), conf);
     Assert.assertEquals(d2, selected);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
index 4bc2d23..538d27d 100644
--- a/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
+++ b/lens-cube/src/test/java/org/apache/lens/driver/cube/TestRewriting.java
@@ -31,10 +31,11 @@ import org.apache.lens.api.LensException;
 import org.apache.lens.cube.parse.CubeQueryContext;
 import org.apache.lens.cube.parse.CubeQueryRewriter;
 import org.apache.lens.cube.parse.HQLParser;
-import org.apache.lens.driver.cube.RewriteUtil;
 import org.apache.lens.server.api.driver.LensDriver;
 
 import static org.mockito.Matchers.any;
+
+import org.apache.lens.server.api.driver.MockDriver;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-cube/src/test/resources/hive-site.xml
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/resources/hive-site.xml b/lens-cube/src/test/resources/hive-site.xml
index 5624f03..b450ec2 100644
--- a/lens-cube/src/test/resources/hive-site.xml
+++ b/lens-cube/src/test/resources/hive-site.xml
@@ -25,7 +25,7 @@
 
 <property>
   <name>lens.server.drivers</name>
-  <value>org.apache.lens.driver.cube.MockDriver</value>
+  <value>org.apache.lens.server.api.driver.MockDriver</value>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MinQueryCostSelector.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MinQueryCostSelector.java b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MinQueryCostSelector.java
index a280c1b..1377d81 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MinQueryCostSelector.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MinQueryCostSelector.java
@@ -20,6 +20,7 @@ package org.apache.lens.server.api.driver;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.lens.api.LensException;
+import org.apache.lens.server.api.query.AbstractQueryContext;
 import org.apache.log4j.Logger;
 
 import java.util.Collection;
@@ -29,55 +30,33 @@ import java.util.Map;
 
 public class MinQueryCostSelector implements DriverSelector {
   public static final Logger LOG = Logger.getLogger(MinQueryCostSelector.class);
-
   /**
    * Returns the driver that has the minimum query cost.
    *
-   * @param drivers
-   *          the drivers
-   * @param driverQueries
-   *          the driver queries
+   * @param ctx
+   *          the context
    * @param conf
    *          the conf
    * @return the lens driver
    */
   @Override
-  public LensDriver select(Collection<LensDriver> drivers, final Map<LensDriver, String> driverQueries,
+  public LensDriver select(final AbstractQueryContext ctx,
     final Configuration conf) {
-    return Collections.min(drivers, new Comparator<LensDriver>() {
+    return Collections.min(ctx.getDriverQueries().keySet(), new Comparator<LensDriver>() {
       @Override
       public int compare(LensDriver d1, LensDriver d2) {
-        DriverQueryPlan c1 = null;
-        DriverQueryPlan c2 = null;
-        // Handle cases where the queries can be null because the storages are not
-        // supported.
-        if (driverQueries.get(d1) == null) {
-          return 1;
-        }
-        if (driverQueries.get(d2) == null) {
-          return -1;
-        }
-        try {
-          c1 = d1.explain(driverQueries.get(d1), conf);
-        } catch (LensException e) {
-          LOG.warn("Explain query:" + driverQueries.get(d1) + " on Driver:" + d1.getClass().getSimpleName()
-            + " failed", e);
-        }
-        try {
-          c2 = d2.explain(driverQueries.get(d2), conf);
-        } catch (LensException e) {
-          LOG.warn("Explain query:" + driverQueries.get(d2) + " on Driver:" + d2.getClass().getSimpleName()
-            + " failed", e);
-        }
-        if (c1 == null && c2 == null) {
-          return 0;
-        } else if (c1 == null && c2 != null) {
-          return 1;
-        } else if (c1 != null && c2 == null) {
-          return -1;
-        }
-        return c1.getCost().compareTo(c2.getCost());
+        return comparePlans(ctx.getDriverQueryPlans().get(d1), ctx.getDriverQueryPlans().get(d2));
       }
     });
   }
+  int comparePlans(DriverQueryPlan c1, DriverQueryPlan c2) {
+    if (c1 == null && c2 == null) {
+      return 0;
+    } else if (c1 == null && c2 != null) {
+      return 1;
+    } else if (c1 != null && c2 == null) {
+      return -1;
+    }
+    return c1.getCost().compareTo(c2.getCost());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockDriver.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockDriver.java b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockDriver.java
new file mode 100644
index 0000000..8487058
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockDriver.java
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.driver;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.service.cli.ColumnDescriptor;
+import org.apache.lens.api.LensException;
+import org.apache.lens.api.query.QueryCost;
+import org.apache.lens.api.query.QueryHandle;
+import org.apache.lens.api.query.QueryPrepareHandle;
+import org.apache.lens.api.query.ResultRow;
+import org.apache.lens.server.api.driver.*;
+import org.apache.lens.server.api.driver.DriverQueryStatus.DriverQueryState;
+import org.apache.lens.server.api.events.LensEventListener;
+import org.apache.lens.server.api.query.PreparedQueryContext;
+import org.apache.lens.server.api.query.QueryContext;
+
+/**
+ * The Class MockDriver.
+ */
+public class MockDriver implements LensDriver {
+
+  /** The conf. */
+  Configuration conf;
+
+  /** The query. */
+  String query;
+
+  /** The io test val. */
+  private int ioTestVal = -1;
+
+  /**
+   * Instantiates a new mock driver.
+   */
+  public MockDriver() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#configure(org.apache.hadoop.conf.Configuration)
+   */
+  @Override
+  public void configure(Configuration conf) throws LensException {
+    this.conf = conf;
+    ioTestVal = conf.getInt("mock.driver.test.val", -1);
+  }
+
+  /**
+   * The Class MockQueryPlan.
+   */
+  public static class MockQueryPlan extends DriverQueryPlan {
+
+    /** The query. */
+    String query;
+
+    /**
+     * Instantiates a new mock query plan.
+     *
+     * @param query
+     *          the query
+     */
+    MockQueryPlan(String query) {
+      this.query = query;
+      setPrepareHandle(new QueryPrepareHandle(UUID.randomUUID()));
+      tableWeights.put("table1", 1.0);
+      tableWeights.put("table2", 2.0);
+      tableWeights.put("table3", 3.0);
+    }
+
+    @Getter @Setter
+    private Map<String, List<String>> partitions;
+
+//    public Map<String, List<String>> getPartitions() {
+//      return partitions;
+//    }
+//    public void setPartitions(Map<String, List<String>> partitions) {
+//      this.partitions = partitions;
+//    }
+
+    @Override
+    public String getPlan() {
+      return query;
+    }
+
+    @Override
+    public QueryCost getCost() {
+      return new QueryCost(0L, 0.0);
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#explain(java.lang.String, org.apache.hadoop.conf.Configuration)
+   */
+  @Override
+  public DriverQueryPlan explain(String query, Configuration conf) throws LensException {
+    return new MockQueryPlan(query);
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#updateStatus(org.apache.lens.server.api.query.QueryContext)
+   */
+  @Override
+  public void updateStatus(QueryContext context) throws LensException {
+    context.getDriverStatus().setProgress(1.0);
+    context.getDriverStatus().setStatusMessage("Done");
+    context.getDriverStatus().setState(DriverQueryState.SUCCESSFUL);
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#cancelQuery(org.apache.lens.api.query.QueryHandle)
+   */
+  @Override
+  public boolean cancelQuery(QueryHandle handle) throws LensException {
+    return false;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#closeQuery(org.apache.lens.api.query.QueryHandle)
+   */
+  @Override
+  public void closeQuery(QueryHandle handle) throws LensException {
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#close()
+   */
+  @Override
+  public void close() throws LensException {
+  }
+
+  /**
+   * Add a listener for driver events.
+   *
+   * @param driverEventListener
+   *          the driver event listener
+   */
+  @Override
+  public void registerDriverEventListener(LensEventListener<DriverEvent> driverEventListener) {
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#prepare(org.apache.lens.server.api.query.PreparedQueryContext)
+   */
+  @Override
+  public void prepare(PreparedQueryContext pContext) throws LensException {
+    // TODO Auto-generated method stub
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.lens.server.api.driver.LensDriver#explainAndPrepare(org.apache.lens.server.api.query.PreparedQueryContext
+   * )
+   */
+  @Override
+  public DriverQueryPlan explainAndPrepare(PreparedQueryContext pContext) throws LensException {
+    DriverQueryPlan p = new MockQueryPlan(pContext.getDriverQuery());
+    p.setPrepareHandle(pContext.getPrepareHandle());
+    return p;
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#closePreparedQuery(org.apache.lens.api.query.QueryPrepareHandle)
+   */
+  @Override
+  public void closePreparedQuery(QueryPrepareHandle handle) throws LensException {
+    // TODO Auto-generated method stub
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#execute(org.apache.lens.server.api.query.QueryContext)
+   */
+  @Override
+  public LensResultSet execute(QueryContext context) throws LensException {
+    this.query = context.getDriverQuery();
+    return new PersistentResultSet() {
+
+      @Override
+      public int size() throws LensException {
+        // TODO Auto-generated method stub
+        return 0;
+      }
+
+      @Override
+      public LensResultSetMetadata getMetadata() throws LensException {
+        // TODO Auto-generated method stub
+        return new LensResultSetMetadata() {
+
+          @Override
+          public List<ColumnDescriptor> getColumns() {
+            // TODO Auto-generated method stub
+            return null;
+          }
+        };
+      }
+
+      @Override
+      public String getOutputPath() throws LensException {
+        // TODO Auto-generated method stub
+        return null;
+      }
+    };
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#executeAsync(org.apache.lens.server.api.query.QueryContext)
+   */
+  @Override
+  public void executeAsync(QueryContext context) throws LensException {
+    this.query = context.getDriverQuery();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#fetchResultSet(org.apache.lens.server.api.query.QueryContext)
+   */
+  @Override
+  public LensResultSet fetchResultSet(QueryContext context) throws LensException {
+    return new InMemoryResultSet() {
+
+      @Override
+      public int size() throws LensException {
+        // TODO Auto-generated method stub
+        return 0;
+      }
+
+      @Override
+      public LensResultSetMetadata getMetadata() throws LensException {
+        return new LensResultSetMetadata() {
+
+          @Override
+          public List<ColumnDescriptor> getColumns() {
+            // TODO Auto-generated method stub
+            return null;
+          }
+        };
+      }
+
+      @Override
+      public void setFetchSize(int size) throws LensException {
+        // TODO Auto-generated method stub
+
+      }
+
+      @Override
+      public ResultRow next() throws LensException {
+        // TODO Auto-generated method stub
+        return null;
+      }
+
+      @Override
+      public boolean hasNext() throws LensException {
+        // TODO Auto-generated method stub
+        return false;
+      }
+    };
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see org.apache.lens.server.api.driver.LensDriver#closeResultSet(org.apache.lens.api.query.QueryHandle)
+   */
+  @Override
+  public void closeResultSet(QueryHandle handle) throws LensException {
+    // TODO Auto-generated method stub
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.lens.server.api.driver.LensDriver#registerForCompletionNotification(org.apache.lens.api.query.QueryHandle
+   * , long, org.apache.lens.server.api.driver.QueryCompletionListener)
+   */
+  @Override
+  public void registerForCompletionNotification(QueryHandle handle, long timeoutMillis, QueryCompletionListener listener)
+      throws LensException {
+    // TODO Auto-generated method stub
+
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.io.Externalizable#readExternal(java.io.ObjectInput)
+   */
+  @Override
+  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+    ioTestVal = in.readInt();
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see java.io.Externalizable#writeExternal(java.io.ObjectOutput)
+   */
+  @Override
+  public void writeExternal(ObjectOutput out) throws IOException {
+    out.writeInt(ioTestVal);
+  }
+
+  public int getTestIOVal() {
+    return ioTestVal;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockFailDriver.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockFailDriver.java b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockFailDriver.java
new file mode 100644
index 0000000..3346701
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockFailDriver.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.driver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.lens.api.LensException;
+import org.apache.lens.server.api.driver.DriverQueryPlan;
+import org.apache.lens.server.api.driver.MockDriver;
+
+
+public class MockFailDriver extends MockDriver {
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.lens.server.api.driver.MockDriver#explain(java.lang.String, org.apache.hadoop.conf.Configuration)
+   */
+  public DriverQueryPlan explain(String query, Configuration conf) throws LensException {
+    throw new LensException("failing!");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/180f6b41/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockQueryContext.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockQueryContext.java b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockQueryContext.java
new file mode 100644
index 0000000..c4d6b6d
--- /dev/null
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/driver/MockQueryContext.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.server.api.driver;
+
+import java.util.Map;
+
+import org.apache.lens.api.LensException;
+import org.apache.lens.server.api.query.AbstractQueryContext;
+
+public class MockQueryContext extends AbstractQueryContext {
+  public MockQueryContext(Map<LensDriver, String> driverQueries) throws LensException {
+    setDriverQueriesAndPlans(driverQueries);
+  }
+}