You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by mm...@apache.org on 2018/06/13 23:00:46 UTC

[2/2] calcite git commit: [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

[CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

Better test coverage for unit tests using Fongo in-memory implementation of Mongo API.
New code will decide (at runtime) what connection to make: fongo vs mongo. Identical tests will be run against
both databases (depending on maven profile surefire vs failsafe)

Close apache/calcite#723


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/dcf396a5
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/dcf396a5
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/dcf396a5

Branch: refs/heads/master
Commit: dcf396a5ca92ee0714c28d9a3c310f53f46220ab
Parents: f84a3eb
Author: Andrei Sereda <an...@nospam.com>
Authored: Tue Jun 5 16:23:56 2018 -0400
Committer: Michael Mior <mm...@uwaterloo.ca>
Committed: Wed Jun 13 18:59:15 2018 -0400

----------------------------------------------------------------------
 mongodb/pom.xml                                 |  10 +-
 .../adapter/mongodb/MongoAdapterTest.java       | 794 +++++++++++++++---
 .../adapter/mongodb/MongoDatabaseRule.java      |  83 ++
 .../org/apache/calcite/test/MongoAdapterIT.java | 830 +------------------
 .../apache/calcite/test/MongoAssertions.java    | 101 +++
 .../test/resources/mongo-foodmart-model.json    | 221 -----
 mongodb/src/test/resources/mongo-model.json     |  70 ++
 .../src/test/resources/mongo-zips-model.json    |  41 -
 mongodb/src/test/resources/zips-mini.json       | 149 ++++
 pom.xml                                         |   9 +
 10 files changed, 1132 insertions(+), 1176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/mongodb/pom.xml b/mongodb/pom.xml
index 25c0642..63cc147 100644
--- a/mongodb/pom.xml
+++ b/mongodb/pom.xml
@@ -71,6 +71,11 @@ limitations under the License.
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>net.hydromatic</groupId>
+      <artifactId>foodmart-data-json</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.mongodb</groupId>
       <artifactId>mongo-java-driver</artifactId>
     </dependency>
@@ -78,11 +83,6 @@ limitations under the License.
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
index b6ca4f3..a4061e4 100644
--- a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
+++ b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
@@ -16,143 +16,749 @@
  */
 package org.apache.calcite.adapter.mongodb;
 
-import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.MongoAssertions;
 
-import com.github.fakemongo.junit.FongoRule;
+
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.io.LineProcessor;
+import com.google.common.io.Resources;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 
+import net.hydromatic.foodmart.data.json.FoodmartJson;
+
+import org.bson.BsonDateTime;
 import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
 import org.bson.Document;
 
-import org.junit.Before;
+import org.hamcrest.CoreMatchers;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Ignore;
-import org.junit.Rule;
 import org.junit.Test;
 
-import java.sql.Connection;
-import java.sql.DriverManager;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
 
 /**
- * Tests current adapter using in-memory (fake) implementation of Mongo API:
- * <a href="https://github.com/fakemongo/fongo">Fongo</a>.
- *
+ * Testing mongo adapter functionality. By default runs with
+ * <a href="https://github.com/fakemongo/fongo">Fongo</a> unless {@code IT} maven profile is enabled
+ * (via {@code $ mvn -Pit install}).
  */
-public class MongoAdapterTest {
+public class MongoAdapterTest implements SchemaFactory {
+
+  /** Connection factory based on the "mongo-zips" model. */
+  protected static final URL MODEL = MongoAdapterTest.class.getResource("/mongo-model.json");
+
+  /** Number of records in local file */
+  protected static final int ZIPS_SIZE = 149;
+
+  @ClassRule
+  public static final MongoDatabaseRule RULE = MongoDatabaseRule.create();
+
+  private static MongoSchema schema;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    MongoDatabase database = RULE.database();
+
+    populate(database.getCollection("zips"), MongoAdapterTest.class.getResource("/zips-mini.json"));
+    populate(database.getCollection("store"), FoodmartJson.class.getResource("/store.json"));
+    populate(database.getCollection("warehouse"),
+            FoodmartJson.class.getResource("/warehouse.json"));
 
-  @Rule
-  public final FongoRule rule = new FongoRule();
+    // Manually insert data for data-time test.
+    MongoCollection<BsonDocument> datatypes =  database.getCollection("datatypes")
+            .withDocumentClass(BsonDocument.class);
+    if (datatypes.count() > 0) {
+      datatypes.deleteMany(new BsonDocument());
+    }
+    BsonDocument doc = new BsonDocument();
+    Date date = new SimpleDateFormat("yyyy-MM-dd", Locale.getDefault()).parse("2012-09-05");
+    doc.put("date", new BsonDateTime(date.getTime()));
+    doc.put("value", new BsonInt32(1231));
+    doc.put("ownerId", new BsonString("531e7789e4b0853ddb861313"));
+    datatypes.insertOne(doc);
 
-  private MongoDatabase mongoDb;
-  private MongoCollection<Document> zips;
+    schema = new MongoSchema(database);
+  }
+
+  private static void populate(MongoCollection<Document> collection, URL resource)
+          throws IOException {
+    Preconditions.checkNotNull(collection, "collection");
+
+    if (collection.count() > 0) {
+      // delete any existing documents (run from a clean set)
+      collection.deleteMany(new BsonDocument());
+    }
 
-  @Before
-  public void setUp() throws Exception {
-    mongoDb = rule.getDatabase(getClass().getSimpleName());
-    zips = mongoDb.getCollection("zips");
+    MongoCollection<BsonDocument> bsonCollection = collection.withDocumentClass(BsonDocument.class);
+    Resources.readLines(resource, StandardCharsets.UTF_8, new LineProcessor<Void>() {
+      @Override public boolean processLine(String line) throws IOException {
+        bsonCollection.insertOne(BsonDocument.parse(line));
+        return true;
+      }
+
+      @Override public Void getResult() {
+        return null;
+      }
+    });
   }
 
   /**
-   * Handcrafted connection where we manually added {@link MongoSchema}
+   *  Returns always the same schema to avoid initialization costs.
    */
-  private CalciteAssert.ConnectionFactory newConnectionFactory() {
-    return new CalciteAssert.ConnectionFactory() {
-      @Override public Connection createConnection() throws SQLException {
-        Connection connection = DriverManager.getConnection("jdbc:calcite:");
-        final SchemaPlus root = connection.unwrap(CalciteConnection.class).getRootSchema();
-        root.add("mongo", new MongoSchema(mongoDb));
-        return connection;
-      }
-    };
+  @Override public Schema create(SchemaPlus parentSchema, String name,
+                                 Map<String, Object> operand) {
+    return schema;
+  }
+
+  private CalciteAssert.AssertThat assertModel(String model) {
+    // ensure that Schema from this instance is being used
+    model = model.replace(MongoSchemaFactory.class.getName(), MongoAdapterTest.class.getName());
+
+    return CalciteAssert.that()
+            .withModel(model);
+  }
+
+  private CalciteAssert.AssertThat assertModel(URL url) {
+    Preconditions.checkNotNull(url, "url");
+    try {
+      return assertModel(Resources.toString(url, StandardCharsets.UTF_8));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
   }
 
   @Test
-  public void single() {
-    zips.insertOne(new Document());
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select * from \"mongo\".\"zips\"")
+  public void testSort() {
+    assertModel(MODEL)
+            .query("select * from zips order by state")
+            .returnsCount(ZIPS_SIZE)
+            .explainContains("PLAN=MongoToEnumerableConverter\n"
+                    + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
+                    + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+                    + "      MongoTableScan(table=[[mongo_raw, zips]])");
+  }
+
+  @Test public void testSortLimit() {
+    assertModel(MODEL)
+            .query("select state, id from zips\n"
+                    + "order by state, id offset 2 rows fetch next 3 rows only")
+            .returns("STATE=AK; ID=99801\n"
+                    + "STATE=AL; ID=35215\n"
+                    + "STATE=AL; ID=35401\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state', ID: '$_id'}}",
+                            "{$sort: {STATE: 1, ID: 1}}",
+                            "{$skip: 2}",
+                            "{$limit: 3}"));
+  }
+
+  @Test public void testOffsetLimit() {
+    assertModel(MODEL)
+            .query("select state, id from zips\n"
+                    + "offset 2 fetch next 3 rows only")
             .runs()
-            .returnsCount(1);
+            .queryContains(
+                    mongoChecker(
+                            "{$skip: 2}",
+                            "{$limit: 3}",
+                            "{$project: {STATE: '$state', ID: '$_id'}}"));
   }
 
-  @Test
-  public void empty() {
-    // for some reason fongo doesn't list collection if it was unused
-    zips.insertOne(new Document());
-    zips.deleteMany(new BsonDocument());
-
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select * from \"mongo\".\"zips\"")
+  @Test public void testLimit() {
+    assertModel(MODEL)
+            .query("select state, id from zips\n"
+                    + "fetch next 3 rows only")
             .runs()
-            .returnsCount(0);
+            .queryContains(
+                    mongoChecker(
+                            "{$limit: 3}",
+                            "{$project: {STATE: '$state', ID: '$_id'}}"));
   }
 
-  @Test
-  public void filter() {
-    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
-    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
+  @Ignore
+  @Test public void testFilterSort() {
+    // LONGITUDE and LATITUDE are null because of CALCITE-194.
+    Util.discard(Bug.CALCITE_194_FIXED);
+    assertModel(MODEL)
+            .query("select * from zips\n"
+                    + "where city = 'SPRINGFIELD' and id >= '70000'\n"
+                    + "order by state, id")
+            .returns(""
+                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=752; STATE=AR; ID=72157\n"
+                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=1992; STATE=CO; ID=81073\n"
+                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=5597; STATE=LA; ID=70462\n"
+                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=32384; STATE=OR; ID=97477\n"
+                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=27521; STATE=OR; ID=97478\n")
+            .queryContains(
+                    mongoChecker(
+                            "{\n"
+                                    + "  $match: {\n"
+                                    + "    city: \"SPRINGFIELD\",\n"
+                                    + "    _id: {\n"
+                                    + "      $gte: \"70000\"\n"
+                                    + "    }\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
+                            "{$sort: {STATE: 1, ID: 1}}"))
+            .explainContains("PLAN=MongoToEnumerableConverter\n"
+                    + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
+                    + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+                    + "      MongoFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
+                    + "        MongoTableScan(table=[[mongo_raw, zips]])");
+  }
 
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
-                    + " where _MAP['STATE'] = 'NY'")
-            .returns("city=New York\n");
+  @Test public void testFilterSortDesc() {
+    assertModel(MODEL)
+            .query("select * from zips\n"
+                    + "where pop BETWEEN 45000 AND 46000\n"
+                    + "order by state desc, pop")
+            .limit(4)
+            .returns("CITY=BECKLEY; LONGITUDE=null; LATITUDE=null; POP=45196; STATE=WV; ID=25801\n"
+                    + "CITY=ROCKERVILLE; LONGITUDE=null; LATITUDE=null; POP=45328; STATE=SD; ID=57701\n"
+                    + "CITY=PAWTUCKET; LONGITUDE=null; LATITUDE=null; POP=45442; STATE=RI; ID=02860\n"
+                    + "CITY=LAWTON; LONGITUDE=null; LATITUDE=null; POP=45542; STATE=OK; ID=73505\n");
+  }
 
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
-                   + " where _MAP['STATE'] = 'DC'")
-            .returns("city=Washington\n");
+  @Ignore("broken; [CALCITE-2115] is logged to fix it")
+  @Test public void testUnionPlan() {
+    assertModel(MODEL)
+            .query("select * from \"sales_fact_1997\"\n"
+                    + "union all\n"
+                    + "select * from \"sales_fact_1998\"")
+            .explainContains("PLAN=EnumerableUnion(all=[true])\n"
+                    + "  MongoToEnumerableConverter\n"
+                    + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
+                    + "      MongoTableScan(table=[[_foodmart, sales_fact_1997]])\n"
+                    + "  MongoToEnumerableConverter\n"
+                    + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
+                    + "      MongoTableScan(table=[[_foodmart, sales_fact_1998]])")
+            .limit(2)
+            .returns(
+                    MongoAssertions.checkResultUnordered(
+                            "product_id=337", "product_id=1512"));
+  }
 
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
-                    + " where _MAP['STATE'] in ('DC', 'NY')")
-            .returns("city=New York\ncity=Washington\n");
+  @Ignore(
+          "java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Double")
+  @Test public void testFilterUnionPlan() {
+    assertModel(MODEL)
+            .query("select * from (\n"
+                    + "  select * from \"sales_fact_1997\"\n"
+                    + "  union all\n"
+                    + "  select * from \"sales_fact_1998\")\n"
+                    + "where \"product_id\" = 1")
+            .runs();
   }
 
-  @Test
-  public void limit() {
-    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
-    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
+  /** Tests that we don't generate multiple constraints on the same column.
+   * MongoDB doesn't like it. If there is an '=', it supersedes all other
+   * operators. */
+  @Test public void testFilterRedundant() {
+    assertModel(MODEL)
+            .query(
+                    "select * from zips where state > 'CA' and state < 'AZ' and state = 'OK'")
+            .runs()
+            .queryContains(
+                    mongoChecker(
+                            "{\n"
+                                    + "  \"$match\": {\n"
+                                    + "    \"state\": \"OK\"\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}"));
+  }
+
+  @Test public void testSelectWhere() {
+    assertModel(MODEL)
+            .query(
+                    "select * from \"warehouse\" where \"warehouse_state_province\" = 'CA'")
+            .explainContains("PLAN=MongoToEnumerableConverter\n"
+                    + "  MongoProject(warehouse_id=[CAST(ITEM($0, 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+                    + "    MongoFilter(condition=[=(CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+                    + "      MongoTableScan(table=[[mongo_raw, warehouse]])")
+            .returns(
+                    MongoAssertions.checkResultUnordered(
+                            "warehouse_id=6; warehouse_state_province=CA",
+                            "warehouse_id=7; warehouse_state_province=CA",
+                            "warehouse_id=14; warehouse_state_province=CA",
+                            "warehouse_id=24; warehouse_state_province=CA"))
+            .queryContains(
+                    // Per https://issues.apache.org/jira/browse/CALCITE-164,
+                    // $match must occur before $project for good performance.
+                    mongoChecker(
+                            "{\n"
+                                    + "  \"$match\": {\n"
+                                    + "    \"warehouse_state_province\": \"CA\"\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {warehouse_id: 1, warehouse_state_province: 1}}"));
+  }
 
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select * from \"mongo\".\"zips\" limit 1")
-            .returnsCount(1);
+  @Test public void testInPlan() {
+    assertModel(MODEL)
+            .query("select \"store_id\", \"store_name\" from \"store\"\n"
+                    + "where \"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
+            .returns(
+                    MongoAssertions.checkResultUnordered(
+                            "store_id=1; store_name=Store 1",
+                            "store_id=3; store_name=Store 3",
+                            "store_id=7; store_name=Store 7",
+                            "store_id=10; store_name=Store 10",
+                            "store_id=11; store_name=Store 11",
+                            "store_id=15; store_name=Store 15",
+                            "store_id=16; store_name=Store 16",
+                            "store_id=24; store_name=Store 24"))
+            .queryContains(
+                    mongoChecker(
+                            "{\n"
+                                    + "  \"$match\": {\n"
+                                    + "    \"$or\": [\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 1\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 10\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 11\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 15\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 16\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 24\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 3\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"store_name\": \"Store 7\"\n"
+                                    + "      }\n"
+                                    + "    ]\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {store_id: 1, store_name: 1}}"));
+  }
 
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select * from \"mongo\".\"zips\" limit 2")
-            .returnsCount(2);
+  /** Simple query based on the "mongo-zips" model. */
+  @Test public void testZips() {
+    assertModel(MODEL)
+            .query("select state, city from zips")
+            .returnsCount(ZIPS_SIZE);
+  }
 
+  @Test public void testCountGroupByEmpty() {
+    assertModel(MODEL)
+            .query("select count(*) from zips")
+            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n", ZIPS_SIZE))
+            .explainContains("PLAN=MongoToEnumerableConverter\n"
+                    + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
+                    + "    MongoTableScan(table=[[mongo_raw, zips]])")
+            .queryContains(
+                    mongoChecker(
+                            "{$group: {_id: {}, 'EXPR$0': {$sum: 1}}}"));
   }
 
-  /**
-   * Following queries are not supported in Mongo adapter :
-   * <pre>
-   * {@code A and (B or C)}
-   * {@code (A or B) and C}
-   * </pre>
+  @Test public void testCountGroupByEmptyMultiplyBy2() {
+    // This operation is not supported by fongo: https://github.com/fakemongo/fongo/issues/152
+    MongoAssertions.assumeRealMongoInstance();
+
+    assertModel(MODEL)
+            .query("select count(*)*2 from zips")
+            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n", ZIPS_SIZE * 2))
+            .queryContains(
+                    mongoChecker(
+                            "{$group: {_id: {}, _0: {$sum: 1}}}",
+                            "{$project: {'EXPR$0': {$multiply: ['$_0', {$literal: 2}]}}}"));
+  }
+
+  @Test public void testGroupByOneColumnNotProjected() {
+    assertModel(MODEL)
+            .query("select count(*) from zips group by state order by 1")
+            .limit(2)
+            .returns("EXPR$0=2\n"
+                    + "EXPR$0=2\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', 'EXPR$0': {$sum: 1}}}",
+                            "{$project: {STATE: '$_id', 'EXPR$0': '$EXPR$0'}}",
+                            "{$project: {'EXPR$0': 1}}",
+                            "{$sort: {EXPR$0: 1}}"));
+  }
+
+  @Test public void testGroupByOneColumn() {
+    assertModel(MODEL)
+            .query(
+                    "select state, count(*) as c from zips group by state order by state")
+            .limit(2)
+            .returns("STATE=AK; C=3\nSTATE=AL; C=3\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
+                            "{$project: {STATE: '$_id', C: '$C'}}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Test public void testGroupByOneColumnReversed() {
+    // Note extra $project compared to testGroupByOneColumn.
+    assertModel(MODEL)
+            .query(
+                    "select count(*) as c, state from zips group by state order by state")
+            .limit(2)
+            .returns("C=3; STATE=AK\nC=3; STATE=AL\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
+                            "{$project: {STATE: '$_id', C: '$C'}}",
+                            "{$project: {C: 1, STATE: 1}}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Test public void testGroupByAvg() {
+    assertModel(MODEL)
+            .query(
+                    "select state, avg(pop) as a from zips group by state order by state")
+            .limit(2)
+            .returns("STATE=AK; A=26856\nSTATE=AL; A=43383\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {POP: '$pop', STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', A: {$avg: '$POP'}}}",
+                            "{$project: {STATE: '$_id', A: '$A'}}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Test public void testGroupByAvgSumCount() {
+    // This operation not supported by fongo: https://github.com/fakemongo/fongo/issues/152
+    MongoAssertions.assumeRealMongoInstance();
+    assertModel(MODEL)
+            .query(
+                    "select state, avg(pop) as a, sum(pop) as s, count(pop) as c from zips group by state order by state")
+            .limit(2)
+            .returns("STATE=AK; A=26856; S=80568; C=3\n"
+                    + "STATE=AL; A=43383; S=130151; C=3\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {POP: '$pop', STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', _1: {$sum: '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
+                            "{$project: {STATE: '$_id', _1: '$_1', _2: '$_2'}}",
+                            "{$sort: {STATE: 1}}",
+                            "{$project: {STATE: 1, A: {$divide: [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S: {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
+  }
+
+  @Test public void testGroupByHaving() {
+    assertModel(MODEL)
+            .query("select state, count(*) as c from zips\n"
+                    + "group by state having count(*) > 2 order by state")
+            .returnsCount(47)
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
+                            "{$project: {STATE: '$_id', C: '$C'}}",
+                            "{\n"
+                                    + "  \"$match\": {\n"
+                                    + "    \"C\": {\n"
+                                    + "      \"$gt\": 2\n"
+                                    + "    }\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
+  @Test public void testGroupByHaving2() {
+    assertModel(MODEL)
+            .query("select state, count(*) as c from zips\n"
+                    + "group by state having sum(pop) > 12000000")
+            .returns("STATE=NY; C=1596\n"
+                    + "STATE=TX; C=1676\n"
+                    + "STATE=FL; C=826\n"
+                    + "STATE=CA; C=1523\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {STATE: '$state', POP: '$pop'}}",
+                            "{$group: {_id: '$STATE', C: {$sum: 1}, _2: {$sum: '$POP'}}}",
+                            "{$project: {STATE: '$_id', C: '$C', _2: '$_2'}}",
+                            "{\n"
+                                    + "  $match: {\n"
+                                    + "    _2: {\n"
+                                    + "      $gt: 12000000\n"
+                                    + "    }\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {STATE: 1, C: 1}}"));
+  }
+
+  @Test public void testGroupByMinMaxSum() {
+    assertModel(MODEL)
+            .query("select count(*) as c, state,\n"
+                    + " min(pop) as min_pop, max(pop) as max_pop, sum(pop) as sum_pop\n"
+                    + "from zips group by state order by state")
+            .limit(2)
+            .returns("C=3; STATE=AK; MIN_POP=23238; MAX_POP=32383; SUM_POP=80568\n"
+                    + "C=3; STATE=AL; MIN_POP=42124; MAX_POP=44165; SUM_POP=130151\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {POP: '$pop', STATE: '$state'}}",
+                            "{$group: {_id: '$STATE', C: {$sum: 1}, MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum: '$POP'}}}",
+                            "{$project: {STATE: '$_id', C: '$C', MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
+                            "{$project: {C: 1, STATE: 1, MIN_POP: 1, MAX_POP: 1, SUM_POP: 1}}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Test public void testGroupComposite() {
+    assertModel(MODEL)
+            .query("select count(*) as c, state, city from zips\n"
+                    + "group by state, city\n"
+                    + "order by c desc, city\n"
+                    + "limit 2")
+            .returns("C=1; STATE=SD; CITY=ABERDEEN\n"
+                      + "C=1; STATE=SC; CITY=AIKEN\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {CITY: '$city', STATE: '$state'}}",
+                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}, C: {$sum: 1}}}",
+                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE', C: '$C'}}",
+                            "{$sort: {C: -1, CITY: 1}}",
+                            "{$limit: 2}",
+                            "{$project: {C: 1, STATE: 1, CITY: 1}}"));
+  }
+
+  @Ignore("broken; [CALCITE-2115] is logged to fix it")
+  @Test public void testDistinctCount() {
+    assertModel(MODEL)
+            .query("select state, count(distinct city) as cdc from zips\n"
+                    + "where state in ('CA', 'TX') group by state order by state")
+            .returns("STATE=CA; CDC=1072\n"
+                    + "STATE=TX; CDC=1233\n")
+            .queryContains(
+                    mongoChecker(
+                            "{\n"
+                                    + "  \"$match\": {\n"
+                                    + "    \"$or\": [\n"
+                                    + "      {\n"
+                                    + "        \"state\": \"CA\"\n"
+                                    + "      },\n"
+                                    + "      {\n"
+                                    + "        \"state\": \"TX\"\n"
+                                    + "      }\n"
+                                    + "    ]\n"
+                                    + "  }\n"
+                                    + "}",
+                            "{$project: {CITY: '$city', STATE: '$state'}}",
+                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
+                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
+                            "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
+                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
+                            "{$sort: {STATE: 1}}"));
+  }
+
+  @Test public void testDistinctCountOrderBy() {
+    // java.lang.ClassCastException: com.mongodb.BasicDBObject cannot be cast to java.lang.Number
+    // https://github.com/fakemongo/fongo/issues/152
+    MongoAssertions.assumeRealMongoInstance();
+    assertModel(MODEL)
+            .query("select state, count(distinct city) as cdc\n"
+                    + "from zips\n"
+                    + "group by state\n"
+                    + "order by cdc desc limit 5")
+            .returns("STATE=VA; CDC=3\n"
+                    + "STATE=NY; CDC=3\n"
+                    + "STATE=SC; CDC=3\n"
+                    + "STATE=RI; CDC=3\n"
+                    + "STATE=WV; CDC=3\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {CITY: '$city', STATE: '$state'}}",
+                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
+                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
+                            "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
+                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
+                            "{$sort: {CDC: -1}}",
+                            "{$limit: 5}"));
+  }
+
+  @Ignore("broken; [CALCITE-2115] is logged to fix it")
+  @Test public void testProject() {
+    assertModel(MODEL)
+            .query("select state, city, 0 as zero from zips order by state, city")
+            .limit(2)
+            .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
+                    + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
+            .queryContains(
+                    mongoChecker(
+                            "{$project: {CITY: '$city', STATE: '$state'}}",
+                            "{$sort: {STATE: 1, CITY: 1}}",
+                            "{$project: {STATE: 1, CITY: 1, ZERO: {$literal: 0}}}"));
+  }
+
+  @Test public void testFilter() {
+    assertModel(MODEL)
+            .query("select state, city from zips where state = 'CA'")
+            .limit(2)
+            .returns("STATE=CA; CITY=LOS ANGELES\n"
+                    + "STATE=CA; CITY=BELL GARDENS\n")
+            .explainContains("PLAN=MongoToEnumerableConverter\n"
+                    + "  MongoProject(STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+                    + "    MongoFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+                    + "      MongoTableScan(table=[[mongo_raw, zips]])");
+  }
+
+  /** MongoDB's predicates are handed (they can only accept literals on the
+   * right-hand size) so it's worth testing that we handle them right both
+   * ways around. */
+  @Test public void testFilterReversed() {
+    assertModel(MODEL)
+            .query("select state, city from zips where 'WI' < state")
+            .limit(2)
+            .returns("STATE=WV; CITY=BECKLEY\nSTATE=WV; CITY=ELM GROVE\n");
+
+    assertModel(MODEL)
+            .query("select state, city from zips where state > 'WI'")
+            .limit(2)
+            .returns("STATE=WV; CITY=BECKLEY\n"
+                    + "STATE=WV; CITY=ELM GROVE\n");
+  }
+
+  /** MongoDB's predicates are handed (they can only accept literals on the
+   * right-hand size) so it's worth testing that we handle them right both
+   * ways around.
    *
-   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2331">[CALCITE-2331]</a>
-   */
-  @Ignore("broken; [CALCITE-2331] is logged to fix it")
-  @Test
-  public void validateCALCITE2331() {
-    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
-    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
-
-    CalciteAssert.that()
-            .with(newConnectionFactory())
-            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
-                    + " where _MAP['STATE'] in ('DC', 'NY') and _MAP['CITY'] = 'New York'")
-            .returns("city=New York\n");
+   * <p>Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-740">[CALCITE-740]
+   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>. */
+  @Test public void testFilterPair() {
+    final int gt9k = 148;
+    final int lt9k = 1;
+    final int gt8k = 148;
+    final int lt8k = 1;
+    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
+    checkPredicate(gt9k, "where pop > 9000");
+    checkPredicate(lt9k, "where pop < 9000");
+    checkPredicate(gt8k, "where pop > 8000");
+    checkPredicate(lt8k, "where pop < 8000");
+    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
+    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
+    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
+    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
+  }
+
+  private void checkPredicate(int expected, String q) {
+    assertModel(MODEL)
+            .query("select count(*) as c from zips\n"
+                    + q)
+            .returns("C=" + expected + "\n");
+    assertModel(MODEL)
+            .query("select * from zips\n"
+                    + q)
+            .returnsCount(expected);
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-286">[CALCITE-286]
+   * Error casting MongoDB date</a>. */
+  @Test public void testDate() {
+    // Assumes that you have created the following collection before running
+    // this test:
+    //
+    // $ mongo
+    // > use test
+    // switched to db test
+    // > db.createCollection("datatypes")
+    // { "ok" : 1 }
+    // > db.datatypes.insert( {
+    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
+    //     "_class" : "com.ericblue.Test",
+    //     "date" : ISODate("2012-09-05T07:00:00Z"),
+    //     "value" : 1231,
+    //     "ownerId" : "531e7789e4b0853ddb861313"
+    //   } )
+    assertModel("{\n"
+            + "  version: '1.0',\n"
+            + "  defaultSchema: 'test',\n"
+            + "   schemas: [\n"
+            + "     {\n"
+            + "       type: 'custom',\n"
+            + "       name: 'test',\n"
+            + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
+            + "       operand: {\n"
+            + "         host: 'localhost',\n"
+            + "         database: 'test'\n"
+            + "       }\n"
+            + "     }\n"
+            + "   ]\n"
+            + "}")
+            .query("select cast(_MAP['date'] as DATE) from \"datatypes\"")
+            .returnsUnordered("EXPR$0=2012-09-05");
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-665">[CALCITE-665]
+   * ClassCastException in MongoDB adapter</a>. */
+  @Test public void testCountViaInt() {
+    assertModel(MODEL)
+        .query("select count(*) from zips")
+        .returns(
+            new Function<ResultSet, Void>() {
+              public Void apply(ResultSet input) {
+                try {
+                  Assert.assertThat(input.next(), CoreMatchers.is(true));
+                  Assert.assertThat(input.getInt(1), CoreMatchers.is(ZIPS_SIZE));
+                  return null;
+                } catch (SQLException e) {
+                  throw new RuntimeException(e);
+                }
+              }
+            });
+  }
+
+  /** Returns a function that checks that a particular MongoDB pipeline is
+   * generated to implement a query. */
+  private static Function<List, Void> mongoChecker(final String... strings) {
+    return new Function<List, Void>() {
+      public Void apply(List actual) {
+        Object[] actualArray =
+                actual == null || actual.isEmpty()
+                        ? null
+                        : ((List) actual.get(0)).toArray();
+        CalciteAssert.assertArrayEqual("expected MongoDB query not found",
+                strings, actualArray);
+        return null;
+      }
+    };
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
new file mode 100644
index 0000000..90bd759
--- /dev/null
+++ b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
@@ -0,0 +1,83 @@
+/*
+ * 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.calcite.adapter.mongodb;
+
+import org.apache.calcite.test.MongoAssertions;
+
+import com.github.fakemongo.Fongo;
+
+import com.google.common.base.Preconditions;
+
+import com.mongodb.MongoClient;
+import com.mongodb.client.MongoDatabase;
+
+import org.junit.rules.ExternalResource;
+
+/**
+ * Instantiates new connection to fongo (or mongo) database depending on current profile
+ * (unit or integration tests).
+ *
+ * By default, this rule is executed as part of a unit test and in-memory database
+ * <a href="https://github.com/fakemongo/fongo">fongo</a> is used.
+ *
+ * <p>However, if maven profile is set to {@code IT} (eg. via command line
+ * {@code $ mvn -Pit install}) this rule will connect to existing (external)
+ * mongo instance ({@code localhost})</p>
+ *
+ */
+class MongoDatabaseRule extends ExternalResource {
+
+  private static final String DB_NAME = "test";
+
+  private final MongoDatabase database;
+  private final MongoClient client;
+
+  private MongoDatabaseRule(MongoClient client) {
+    this.client = Preconditions.checkNotNull(client, "client");
+    this.database = client.getDatabase(DB_NAME);
+  }
+
+  /**
+   * Create an instance based on current maven profile (as defined by {@code -Pit}).
+   */
+  static MongoDatabaseRule create() {
+    final MongoClient client;
+    if (MongoAssertions.useMongo()) {
+      // use to real client (connects to mongo)
+      client = new MongoClient();
+    } else if (MongoAssertions.useFongo()) {
+      // in-memory DB (fake Mongo)
+      client = new Fongo(MongoDatabaseRule.class.getSimpleName()).getMongo();
+    } else {
+      throw new UnsupportedOperationException("I can only connect to Mongo or Fongo instances");
+    }
+
+    return new MongoDatabaseRule(client);
+  }
+
+
+  MongoDatabase database() {
+    return database;
+  }
+
+  @Override protected void after() {
+    client.close();
+  }
+
+}
+
+// End MongoDatabaseRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
index 00ed03a..bda1163 100644
--- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
+++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
@@ -16,829 +16,29 @@
  */
 package org.apache.calcite.test;
 
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.util.Bug;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
+import org.apache.calcite.adapter.mongodb.MongoAdapterTest;
 
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Ordering;
+import org.junit.BeforeClass;
 
-import org.hamcrest.CoreMatchers;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.assertThat;
+import static org.junit.Assume.assumeTrue;
 
 /**
- * Tests for the {@code org.apache.calcite.adapter.mongodb} package.
- *
- * <p>Before calling this test, you need to populate MongoDB, as follows:
+ * Used to trigger integration tests from maven (thus class name is suffixed with {@code IT}).
  *
- * <blockquote><code>
- * git clone https://github.com/vlsi/calcite-test-dataset<br>
- * cd calcite-test-dataset<br>
- * mvn install
- * </code></blockquote>
+ * <p>If you want to run integration tests from IDE manually set
+ * {@code -Dcalcite.integrationTest=true} system property.
  *
- * <p>This will create a virtual machine with MongoDB and "zips" and "foodmart"
- * data sets.
+ * For command line use:
+ * <pre>
+ *     $ mvn install -Pit
+ * </pre>
+ * </p>
  */
-public class MongoAdapterIT {
-  public static final String MONGO_FOODMART_SCHEMA = "     {\n"
-      + "       type: 'custom',\n"
-      + "       name: '_foodmart',\n"
-      + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
-      + "       operand: {\n"
-      + "         host: 'localhost',\n"
-      + "         database: 'foodmart'\n"
-      + "       }\n"
-      + "     },\n"
-      + "     {\n"
-      + "       name: 'foodmart',\n"
-      + "       tables: [\n"
-      + "         {\n"
-      + "           name: 'sales_fact_1997',\n"
-      + "           type: 'view',\n"
-      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double) AS \"product_id\" from \"_foodmart\".\"sales_fact_1997\"'\n"
-      + "         },\n"
-      + "         {\n"
-      + "           name: 'sales_fact_1998',\n"
-      + "           type: 'view',\n"
-      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double) AS \"product_id\" from \"_foodmart\".\"sales_fact_1998\"'\n"
-      + "         },\n"
-      + "         {\n"
-      + "           name: 'store',\n"
-      + "           type: 'view',\n"
-      + "           sql: 'select cast(_MAP[\\'store_id\\'] AS double) AS \"store_id\", cast(_MAP[\\'store_name\\'] AS varchar(20)) AS \"store_name\" from \"_foodmart\".\"store\"'\n"
-      + "         },\n"
-      + "         {\n"
-      + "           name: 'warehouse',\n"
-      + "           type: 'view',\n"
-      + "           sql: 'select cast(_MAP[\\'warehouse_id\\'] AS double) AS \"warehouse_id\", cast(_MAP[\\'warehouse_state_province\\'] AS varchar(20)) AS \"warehouse_state_province\" from \"_foodmart\".\"warehouse\"'\n"
-      + "         }\n"
-      + "       ]\n"
-      + "     }\n";
-
-  public static final String MONGO_FOODMART_MODEL = "{\n"
-      + "  version: '1.0',\n"
-      + "  defaultSchema: 'foodmart',\n"
-      + "   schemas: [\n"
-      + MONGO_FOODMART_SCHEMA
-      + "   ]\n"
-      + "}";
-
-  /** Connection factory based on the "mongo-zips" model. */
-  public static final ImmutableMap<String, String> ZIPS =
-      ImmutableMap.of("model",
-          MongoAdapterIT.class.getResource("/mongo-zips-model.json")
-              .getPath());
-
-  /** Connection factory based on the "mongo-zips" model. */
-  public static final ImmutableMap<String, String> FOODMART =
-      ImmutableMap.of("model",
-          MongoAdapterIT.class.getResource("/mongo-foodmart-model.json")
-              .getPath());
-
-  /** Whether to run Mongo tests. Enabled by default, however test is only
-   * included if "it" profile is activated ({@code -Pit}). To disable,
-   * specify {@code -Dcalcite.test.mongodb=false} on the Java command line. */
-  public static final boolean ENABLED =
-      Util.getBooleanProperty("calcite.test.mongodb", true);
-
-  /** Whether to run this test. */
-  protected boolean enabled() {
-    return ENABLED;
-  }
-
-  /** Returns a function that checks that a particular MongoDB pipeline is
-   * generated to implement a query. */
-  private static Function<List, Void> mongoChecker(final String... strings) {
-    return new Function<List, Void>() {
-      public Void apply(List actual) {
-        Object[] actualArray =
-            actual == null || actual.isEmpty()
-                ? null
-                : ((List) actual.get(0)).toArray();
-        CalciteAssert.assertArrayEqual("expected MongoDB query not found",
-            strings, actualArray);
-        return null;
-      }
-    };
-  }
-
-  /** Similar to {@link CalciteAssert#checkResultUnordered}, but filters strings
-   * before comparing them. */
-  static Function<ResultSet, Void> checkResultUnordered(
-      final String... lines) {
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          final List<String> expectedList =
-              Ordering.natural().immutableSortedCopy(Arrays.asList(lines));
-
-          final List<String> actualList = Lists.newArrayList();
-          CalciteAssert.toStringList(resultSet, actualList);
-          for (int i = 0; i < actualList.size(); i++) {
-            String s = actualList.get(i);
-            actualList.set(i,
-                s.replaceAll("\\.0;", ";").replaceAll("\\.0$", ""));
-          }
-          Collections.sort(actualList);
-
-          assertThat(Ordering.natural().immutableSortedCopy(actualList),
-              equalTo(expectedList));
-          return null;
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  @Test public void testSort() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips order by state")
-        .returnsCount(29353)
-        .explainContains("PLAN=MongoToEnumerableConverter\n"
-            + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
-            + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-            + "      MongoTableScan(table=[[mongo_raw, zips]])");
-  }
-
-  @Test public void testSortLimit() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, id from zips\n"
-            + "order by state, id offset 2 rows fetch next 3 rows only")
-        .returns("STATE=AK; ID=99503\n"
-            + "STATE=AK; ID=99504\n"
-            + "STATE=AK; ID=99505\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state', ID: '$_id'}}",
-                "{$sort: {STATE: 1, ID: 1}}",
-                "{$skip: 2}",
-                "{$limit: 3}"));
-  }
-
-  @Test public void testOffsetLimit() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, id from zips\n"
-            + "offset 2 fetch next 3 rows only")
-        .runs()
-        .queryContains(
-            mongoChecker(
-                "{$skip: 2}",
-                "{$limit: 3}",
-                "{$project: {STATE: '$state', ID: '$_id'}}"));
-  }
-
-  @Test public void testLimit() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, id from zips\n"
-            + "fetch next 3 rows only")
-        .runs()
-        .queryContains(
-            mongoChecker(
-                "{$limit: 3}",
-                "{$project: {STATE: '$state', ID: '$_id'}}"));
-  }
-
-  @Ignore
-  @Test public void testFilterSort() {
-    // LONGITUDE and LATITUDE are null because of CALCITE-194.
-    Util.discard(Bug.CALCITE_194_FIXED);
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips\n"
-            + "where city = 'SPRINGFIELD' and id >= '70000'\n"
-            + "order by state, id")
-        .returns(""
-            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=752; STATE=AR; ID=72157\n"
-            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=1992; STATE=CO; ID=81073\n"
-            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=5597; STATE=LA; ID=70462\n"
-            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=32384; STATE=OR; ID=97477\n"
-            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=27521; STATE=OR; ID=97478\n")
-        .queryContains(
-            mongoChecker(
-                "{\n"
-                    + "  $match: {\n"
-                    + "    city: \"SPRINGFIELD\",\n"
-                    + "    _id: {\n"
-                    + "      $gte: \"70000\"\n"
-                    + "    }\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
-                "{$sort: {STATE: 1, ID: 1}}"))
-        .explainContains("PLAN=MongoToEnumerableConverter\n"
-            + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
-            + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-            + "      MongoFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
-            + "        MongoTableScan(table=[[mongo_raw, zips]])");
-  }
-
-  @Test public void testFilterSortDesc() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips\n"
-            + "where pop BETWEEN 20000 AND 20100\n"
-            + "order by state desc, pop")
-        .limit(4)
-        .returns(""
-            + "CITY=SHERIDAN; LONGITUDE=null; LATITUDE=null; POP=20025; STATE=WY; ID=82801\n"
-            + "CITY=MOUNTLAKE TERRAC; LONGITUDE=null; LATITUDE=null; POP=20059; STATE=WA; ID=98043\n"
-            + "CITY=FALMOUTH; LONGITUDE=null; LATITUDE=null; POP=20039; STATE=VA; ID=22405\n"
-            + "CITY=FORT WORTH; LONGITUDE=null; LATITUDE=null; POP=20012; STATE=TX; ID=76104\n");
-  }
-
-  @Ignore("broken; [CALCITE-2115] is logged to fix it")
-  @Test public void testUnionPlan() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .withModel(MONGO_FOODMART_MODEL)
-        .query("select * from \"sales_fact_1997\"\n"
-            + "union all\n"
-            + "select * from \"sales_fact_1998\"")
-        .explainContains("PLAN=EnumerableUnion(all=[true])\n"
-            + "  MongoToEnumerableConverter\n"
-            + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
-            + "      MongoTableScan(table=[[_foodmart, sales_fact_1997]])\n"
-            + "  MongoToEnumerableConverter\n"
-            + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
-            + "      MongoTableScan(table=[[_foodmart, sales_fact_1998]])")
-        .limit(2)
-        .returns(
-            checkResultUnordered(
-                "product_id=337", "product_id=1512"));
-  }
-
-  @Ignore(
-      "java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Double")
-  @Test public void testFilterUnionPlan() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .withModel(MONGO_FOODMART_MODEL)
-        .query("select * from (\n"
-            + "  select * from \"sales_fact_1997\"\n"
-            + "  union all\n"
-            + "  select * from \"sales_fact_1998\")\n"
-            + "where \"product_id\" = 1")
-        .runs();
-  }
-
-  /** Tests that we don't generate multiple constraints on the same column.
-   * MongoDB doesn't like it. If there is an '=', it supersedes all other
-   * operators. */
-  @Test public void testFilterRedundant() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(
-            "select * from zips where state > 'CA' and state < 'AZ' and state = 'OK'")
-        .runs()
-        .queryContains(
-            mongoChecker(
-                "{\n"
-                    + "  \"$match\": {\n"
-                    + "    \"state\": \"OK\"\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}"));
-  }
-
-  @Test public void testSelectWhere() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .withModel(MONGO_FOODMART_MODEL)
-        .query(
-            "select * from \"warehouse\" where \"warehouse_state_province\" = 'CA'")
-        .explainContains("PLAN=MongoToEnumerableConverter\n"
-            + "  MongoProject(warehouse_id=[CAST(ITEM($0, 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-            + "    MongoFilter(condition=[=(CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
-            + "      MongoTableScan(table=[[_foodmart, warehouse]])")
-        .returns(
-            checkResultUnordered(
-                "warehouse_id=6; warehouse_state_province=CA",
-                "warehouse_id=7; warehouse_state_province=CA",
-                "warehouse_id=14; warehouse_state_province=CA",
-                "warehouse_id=24; warehouse_state_province=CA"))
-        .queryContains(
-            // Per https://issues.apache.org/jira/browse/CALCITE-164,
-            // $match must occur before $project for good performance.
-            mongoChecker(
-                "{\n"
-                    + "  \"$match\": {\n"
-                    + "    \"warehouse_state_province\": \"CA\"\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {warehouse_id: 1, warehouse_state_province: 1}}"));
-  }
-
-  @Test public void testInPlan() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .withModel(MONGO_FOODMART_MODEL)
-        .query("select \"store_id\", \"store_name\" from \"store\"\n"
-            + "where \"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
-        .returns(
-            checkResultUnordered(
-                "store_id=1; store_name=Store 1",
-                "store_id=3; store_name=Store 3",
-                "store_id=7; store_name=Store 7",
-                "store_id=10; store_name=Store 10",
-                "store_id=11; store_name=Store 11",
-                "store_id=15; store_name=Store 15",
-                "store_id=16; store_name=Store 16",
-                "store_id=24; store_name=Store 24"))
-        .queryContains(
-            mongoChecker(
-                "{\n"
-                    + "  \"$match\": {\n"
-                    + "    \"$or\": [\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 1\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 10\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 11\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 15\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 16\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 24\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 3\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"store_name\": \"Store 7\"\n"
-                    + "      }\n"
-                    + "    ]\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {store_id: 1, store_name: 1}}"));
-  }
-
-  /** Simple query based on the "mongo-zips" model. */
-  @Test public void testZips() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, city from zips")
-        .returnsCount(29353);
-  }
-
-  @Test public void testCountGroupByEmpty() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) from zips")
-        .returns("EXPR$0=29353\n")
-        .explainContains("PLAN=MongoToEnumerableConverter\n"
-            + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
-            + "    MongoTableScan(table=[[mongo_raw, zips]])")
-        .queryContains(
-            mongoChecker(
-                "{$group: {_id: {}, 'EXPR$0': {$sum: 1}}}"));
-  }
-
-  @Test public void testCountGroupByEmptyMultiplyBy2() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*)*2 from zips")
-        .returns("EXPR$0=58706\n")
-        .queryContains(
-            mongoChecker(
-                "{$group: {_id: {}, _0: {$sum: 1}}}",
-                "{$project: {'EXPR$0': {$multiply: ['$_0', {$literal: 2}]}}}"));
-  }
-
-  @Test public void testGroupByOneColumnNotProjected() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) from zips group by state order by 1")
-        .limit(2)
-        .returns("EXPR$0=24\n"
-            + "EXPR$0=53\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state'}}",
-                "{$group: {_id: '$STATE', 'EXPR$0': {$sum: 1}}}",
-                "{$project: {STATE: '$_id', 'EXPR$0': '$EXPR$0'}}",
-                "{$project: {'EXPR$0': 1}}",
-                "{$sort: {EXPR$0: 1}}"));
-  }
-
-  @Test public void testGroupByOneColumn() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(
-            "select state, count(*) as c from zips group by state order by state")
-        .limit(2)
-        .returns("STATE=AK; C=195\n"
-            + "STATE=AL; C=567\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state'}}",
-                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
-                "{$project: {STATE: '$_id', C: '$C'}}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Test public void testGroupByOneColumnReversed() {
-    // Note extra $project compared to testGroupByOneColumn.
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(
-            "select count(*) as c, state from zips group by state order by state")
-        .limit(2)
-        .returns("C=195; STATE=AK\n"
-            + "C=567; STATE=AL\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state'}}",
-                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
-                "{$project: {STATE: '$_id', C: '$C'}}",
-                "{$project: {C: 1, STATE: 1}}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Test public void testGroupByAvg() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(
-            "select state, avg(pop) as a from zips group by state order by state")
-        .limit(2)
-        .returns("STATE=AK; A=2793\n"
-            + "STATE=AL; A=7126\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {POP: '$pop', STATE: '$state'}}",
-                "{$group: {_id: '$STATE', A: {$avg: '$POP'}}}",
-                "{$project: {STATE: '$_id', A: '$A'}}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Test public void testGroupByAvgSumCount() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(
-            "select state, avg(pop) as a, sum(pop) as s, count(pop) as c from zips group by state order by state")
-        .limit(2)
-        .returns("STATE=AK; A=2793; S=544698; C=195\n"
-            + "STATE=AL; A=7126; S=4040587; C=567\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {POP: '$pop', STATE: '$state'}}",
-                "{$group: {_id: '$STATE', _1: {$sum: '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
-                "{$project: {STATE: '$_id', _1: '$_1', _2: '$_2'}}",
-                "{$sort: {STATE: 1}}",
-                "{$project: {STATE: 1, A: {$divide: [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S: {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
-  }
-
-  @Test public void testGroupByHaving() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, count(*) as c from zips\n"
-            + "group by state having count(*) > 1500 order by state")
-        .returns("STATE=CA; C=1516\n"
-            + "STATE=NY; C=1595\n"
-            + "STATE=TX; C=1671\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state'}}",
-                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
-                "{$project: {STATE: '$_id', C: '$C'}}",
-                "{\n"
-                    + "  \"$match\": {\n"
-                    + "    \"C\": {\n"
-                    + "      \"$gt\": 1500\n"
-                    + "    }\n"
-                    + "  }\n"
-                    + "}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
-  @Test public void testGroupByHaving2() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, count(*) as c from zips\n"
-            + "group by state having sum(pop) > 12000000")
-        .returns("STATE=NY; C=1596\n"
-            + "STATE=TX; C=1676\n"
-            + "STATE=FL; C=826\n"
-            + "STATE=CA; C=1523\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {STATE: '$state', POP: '$pop'}}",
-                "{$group: {_id: '$STATE', C: {$sum: 1}, _2: {$sum: '$POP'}}}",
-                "{$project: {STATE: '$_id', C: '$C', _2: '$_2'}}",
-                "{\n"
-                    + "  $match: {\n"
-                    + "    _2: {\n"
-                    + "      $gt: 12000000\n"
-                    + "    }\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {STATE: 1, C: 1}}"));
-  }
-
-  @Test public void testGroupByMinMaxSum() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) as c, state,\n"
-            + " min(pop) as min_pop, max(pop) as max_pop, sum(pop) as sum_pop\n"
-            + "from zips group by state order by state")
-        .limit(2)
-        .returns("C=195; STATE=AK; MIN_POP=0; MAX_POP=32383; SUM_POP=544698\n"
-            + "C=567; STATE=AL; MIN_POP=0; MAX_POP=44165; SUM_POP=4040587\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {POP: '$pop', STATE: '$state'}}",
-                "{$group: {_id: '$STATE', C: {$sum: 1}, MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum: '$POP'}}}",
-                "{$project: {STATE: '$_id', C: '$C', MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
-                "{$project: {C: 1, STATE: 1, MIN_POP: 1, MAX_POP: 1, SUM_POP: 1}}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Test public void testGroupComposite() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) as c, state, city from zips\n"
-            + "group by state, city order by c desc limit 2")
-        .returns("C=93; STATE=TX; CITY=HOUSTON\n"
-            + "C=56; STATE=CA; CITY=LOS ANGELES\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {CITY: '$city', STATE: '$state'}}",
-                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}, C: {$sum: 1}}}",
-                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE', C: '$C'}}",
-                "{$sort: {C: -1}}",
-                "{$limit: 2}",
-                "{$project: {C: 1, STATE: 1, CITY: 1}}"));
-  }
-
-  @Ignore("broken; [CALCITE-2115] is logged to fix it")
-  @Test public void testDistinctCount() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, count(distinct city) as cdc from zips\n"
-            + "where state in ('CA', 'TX') group by state order by state")
-        .returns("STATE=CA; CDC=1072\n"
-            + "STATE=TX; CDC=1233\n")
-        .queryContains(
-            mongoChecker(
-                "{\n"
-                    + "  \"$match\": {\n"
-                    + "    \"$or\": [\n"
-                    + "      {\n"
-                    + "        \"state\": \"CA\"\n"
-                    + "      },\n"
-                    + "      {\n"
-                    + "        \"state\": \"TX\"\n"
-                    + "      }\n"
-                    + "    ]\n"
-                    + "  }\n"
-                    + "}",
-                "{$project: {CITY: '$city', STATE: '$state'}}",
-                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
-                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
-                "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
-                "{$project: {STATE: '$_id', CDC: '$CDC'}}",
-                "{$sort: {STATE: 1}}"));
-  }
-
-  @Test public void testDistinctCountOrderBy() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, count(distinct city) as cdc\n"
-            + "from zips\n"
-            + "group by state\n"
-            + "order by cdc desc limit 5")
-        .returns("STATE=NY; CDC=1370\n"
-            + "STATE=PA; CDC=1369\n"
-            + "STATE=TX; CDC=1233\n"
-            + "STATE=IL; CDC=1148\n"
-            + "STATE=CA; CDC=1072\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {CITY: '$city', STATE: '$state'}}",
-                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
-                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
-                "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
-                "{$project: {STATE: '$_id', CDC: '$CDC'}}",
-                "{$sort: {CDC: -1}}",
-                "{$limit: 5}"));
-  }
-
-  @Ignore("broken; [CALCITE-2115] is logged to fix it")
-  @Test public void testProject() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, city, 0 as zero from zips order by state, city")
-        .limit(2)
-        .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
-            + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
-        .queryContains(
-            mongoChecker(
-                "{$project: {CITY: '$city', STATE: '$state'}}",
-                "{$sort: {STATE: 1, CITY: 1}}",
-                "{$project: {STATE: 1, CITY: 1, ZERO: {$literal: 0}}}"));
-  }
-
-  @Test public void testFilter() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, city from zips where state = 'CA'")
-        .limit(2)
-        .returns("STATE=CA; CITY=LOS ANGELES\n"
-            + "STATE=CA; CITY=LOS ANGELES\n")
-        .explainContains("PLAN=MongoToEnumerableConverter\n"
-            + "  MongoProject(STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
-            + "    MongoFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
-            + "      MongoTableScan(table=[[mongo_raw, zips]])");
-  }
-
-  /** MongoDB's predicates are handed (they can only accept literals on the
-   * right-hand size) so it's worth testing that we handle them right both
-   * ways around. */
-  @Test public void testFilterReversed() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, city from zips where 'WI' < state")
-        .limit(2)
-        .returns("STATE=WV; CITY=BLUEWELL\n"
-            + "STATE=WV; CITY=ATHENS\n");
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select state, city from zips where state > 'WI'")
-        .limit(2)
-        .returns("STATE=WV; CITY=BLUEWELL\n"
-            + "STATE=WV; CITY=ATHENS\n");
-  }
-
-  /** MongoDB's predicates are handed (they can only accept literals on the
-   * right-hand size) so it's worth testing that we handle them right both
-   * ways around.
-   *
-   * <p>Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-740">[CALCITE-740]
-   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>. */
-  @Test public void testFilterPair() {
-    final int gt9k = 8125;
-    final int lt9k = 21227;
-    final int gt8k = 8707;
-    final int lt8k = 20645;
-    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
-    checkPredicate(gt9k, "where pop > 9000");
-    checkPredicate(lt9k, "where pop < 9000");
-    checkPredicate(gt8k, "where pop > 8000");
-    checkPredicate(lt8k, "where pop < 8000");
-    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
-    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
-    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
-    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
-  }
-
-  private void checkPredicate(int expected, String q) {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) as c from zips\n"
-            + q)
-        .returns("C=" + expected + "\n");
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips\n"
-            + q)
-        .returnsCount(expected);
-  }
-
-  @Ignore
-  @Test public void testFoodmartQueries() {
-    final List<Pair<String, String>> queries = JdbcTest.getFoodmartQueries();
-    for (Ord<Pair<String, String>> query : Ord.zip(queries)) {
-//      if (query.i != 29) continue;
-      if (query.e.left.contains("agg_")) {
-        continue;
-      }
-      final CalciteAssert.AssertQuery query1 =
-          CalciteAssert.that()
-              .enable(enabled())
-              .with(FOODMART)
-              .query(query.e.left);
-      if (query.e.right != null) {
-        query1.returns(query.e.right);
-      } else {
-        query1.runs();
-      }
-    }
-  }
-
-  /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-286">[CALCITE-286]
-   * Error casting MongoDB date</a>. */
-  @Test public void testDate() {
-    // Assumes that you have created the following collection before running
-    // this test:
-    //
-    // $ mongo
-    // > use test
-    // switched to db test
-    // > db.createCollection("datatypes")
-    // { "ok" : 1 }
-    // > db.datatypes.insert( {
-    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
-    //     "_class" : "com.ericblue.Test",
-    //     "date" : ISODate("2012-09-05T07:00:00Z"),
-    //     "value" : 1231,
-    //     "ownerId" : "531e7789e4b0853ddb861313"
-    //   } )
-    CalciteAssert.that()
-        .enable(enabled())
-        .withModel("{\n"
-            + "  version: '1.0',\n"
-            + "  defaultSchema: 'test',\n"
-            + "   schemas: [\n"
-            + "     {\n"
-            + "       type: 'custom',\n"
-            + "       name: 'test',\n"
-            + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
-            + "       operand: {\n"
-            + "         host: 'localhost',\n"
-            + "         database: 'test'\n"
-            + "       }\n"
-            + "     }\n"
-            + "   ]\n"
-            + "}")
-        .query("select cast(_MAP['date'] as DATE) from \"datatypes\"")
-        .returnsUnordered("EXPR$0=2012-09-05");
-  }
+public class MongoAdapterIT extends MongoAdapterTest {
 
-  /** Test case for
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-665">[CALCITE-665]
-   * ClassCastException in MongoDB adapter</a>. */
-  @Test public void testCountViaInt() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select count(*) from zips")
-        .returns(
-            new Function<ResultSet, Void>() {
-              public Void apply(ResultSet input) {
-                try {
-                  assertThat(input.next(), CoreMatchers.is(true));
-                  assertThat(input.getInt(1), CoreMatchers.is(29353));
-                  return null;
-                } catch (SQLException e) {
-                  throw new RuntimeException(e);
-                }
-              }
-            });
+  @BeforeClass
+  public static void enforceMongo() {
+    assumeTrue(MongoAssertions.useMongo());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
new file mode 100644
index 0000000..028286a
--- /dev/null
+++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
@@ -0,0 +1,101 @@
+/*
+ * 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.calcite.test;
+
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Util class which needs to be in the same package as {@link CalciteAssert}
+ * due to package-private visibility.
+ */
+public class MongoAssertions {
+
+  private MongoAssertions() {}
+
+  /** Similar to {@link CalciteAssert#checkResultUnordered}, but filters strings
+   * before comparing them. */
+  public static Function<ResultSet, Void> checkResultUnordered(
+      final String... lines) {
+    return new Function<ResultSet, Void>() {
+      public Void apply(ResultSet resultSet) {
+        try {
+          final List<String> expectedList =
+              Ordering.natural().immutableSortedCopy(Arrays.asList(lines));
+
+          final List<String> actualList = Lists.newArrayList();
+          CalciteAssert.toStringList(resultSet, actualList);
+          for (int i = 0; i < actualList.size(); i++) {
+            String s = actualList.get(i);
+            actualList.set(i,
+                s.replaceAll("\\.0;", ";").replaceAll("\\.0$", ""));
+          }
+          Collections.sort(actualList);
+
+          assertThat(Ordering.natural().immutableSortedCopy(actualList),
+              equalTo(expectedList));
+          return null;
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  /**
+   * Whether to run Mongo integration tests. Enabled by default, however test is only
+   * included if "it" profile is activated ({@code -Pit}). To disable,
+   * specify {@code -Dcalcite.test.mongodb=false} on the Java command line.
+   */
+  public static boolean useMongo() {
+    return Util.getBooleanProperty("calcite.integrationTest")
+            && Util.getBooleanProperty("calcite.test.mongodb", true);
+  }
+
+  /**
+   * Checks wherever tests should use Fongo instead of Mongo. Opposite of {@link #useMongo()}.
+   */
+  public static boolean useFongo() {
+    return !useMongo();
+  }
+
+
+  /**
+   * Used to skip tests if current instance is not mongo. Some functionalities
+   * are not available in fongo.
+   *
+   * @see <a href="https://github.com/fakemongo/fongo/issues/152">Aggregation with $cond (172)</a>
+   */
+  public static void assumeRealMongoInstance() {
+    assumeTrue("Expect mongo instance", useMongo());
+  }
+}
+
+// End MongoAssertions.java


Re: [2/2] calcite git commit: [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

Posted by Michael Mior <mm...@apache.org>.
Fixing now.
--
Michael Mior
mmior@apache.org


Le jeu. 14 juin 2018 à 10:42, Julian Hyde <jh...@apache.org> a écrit :

> This change introduces a javadoc error:
>
> [ERROR]
> /home/jhyde/regress/calcite/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java:35:
> error: unexpected end tag: </p>
> [ERROR]  * </p>
>
> Please fix ASAP.
>
>
> > On Jun 13, 2018, at 6:00 PM, mmior@apache.org wrote:
> >
> > [CALCITE-2345] Running Unit tests with Fongo and integration tests with
> real mongo instance (Andrei Sereda)
> >
> > Better test coverage for unit tests using Fongo in-memory implementation
> of Mongo API.
> > New code will decide (at runtime) what connection to make: fongo vs
> mongo. Identical tests will be run against
> > both databases (depending on maven profile surefire vs failsafe)
> >
> > Close apache/calcite#723
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
> > Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/dcf396a5
> > Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/dcf396a5
> > Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/dcf396a5
> >
> > Branch: refs/heads/master
> > Commit: dcf396a5ca92ee0714c28d9a3c310f53f46220ab
> > Parents: f84a3eb
> > Author: Andrei Sereda <an...@nospam.com>
> > Authored: Tue Jun 5 16:23:56 2018 -0400
> > Committer: Michael Mior <mm...@uwaterloo.ca>
> > Committed: Wed Jun 13 18:59:15 2018 -0400
> >
> > ----------------------------------------------------------------------
> > mongodb/pom.xml                                 |  10 +-
> > .../adapter/mongodb/MongoAdapterTest.java       | 794 +++++++++++++++---
> > .../adapter/mongodb/MongoDatabaseRule.java      |  83 ++
> > .../org/apache/calcite/test/MongoAdapterIT.java | 830 +------------------
> > .../apache/calcite/test/MongoAssertions.java    | 101 +++
> > .../test/resources/mongo-foodmart-model.json    | 221 -----
> > mongodb/src/test/resources/mongo-model.json     |  70 ++
> > .../src/test/resources/mongo-zips-model.json    |  41 -
> > mongodb/src/test/resources/zips-mini.json       | 149 ++++
> > pom.xml                                         |   9 +
> > 10 files changed, 1132 insertions(+), 1176 deletions(-)
> > ----------------------------------------------------------------------
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/pom.xml
> > ----------------------------------------------------------------------
> > diff --git a/mongodb/pom.xml b/mongodb/pom.xml
> > index 25c0642..63cc147 100644
> > --- a/mongodb/pom.xml
> > +++ b/mongodb/pom.xml
> > @@ -71,6 +71,11 @@ limitations under the License.
> >       <scope>test</scope>
> >     </dependency>
> >     <dependency>
> > +      <groupId>net.hydromatic</groupId>
> > +      <artifactId>foodmart-data-json</artifactId>
> > +      <scope>test</scope>
> > +    </dependency>
> > +    <dependency>
> >       <groupId>org.mongodb</groupId>
> >       <artifactId>mongo-java-driver</artifactId>
> >     </dependency>
> > @@ -78,11 +83,6 @@ limitations under the License.
> >       <groupId>org.slf4j</groupId>
> >       <artifactId>slf4j-api</artifactId>
> >     </dependency>
> > -    <dependency>
> > -      <groupId>org.slf4j</groupId>
> > -      <artifactId>slf4j-log4j12</artifactId>
> > -      <scope>test</scope>
> > -    </dependency>
> >   </dependencies>
> >
> >   <build>
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > index b6ca4f3..a4061e4 100644
> > ---
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > +++
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > @@ -16,143 +16,749 @@
> >  */
> > package org.apache.calcite.adapter.mongodb;
> >
> > -import org.apache.calcite.jdbc.CalciteConnection;
> > +import org.apache.calcite.schema.Schema;
> > +import org.apache.calcite.schema.SchemaFactory;
> > import org.apache.calcite.schema.SchemaPlus;
> > import org.apache.calcite.test.CalciteAssert;
> > +import org.apache.calcite.test.MongoAssertions;
> >
> > -import com.github.fakemongo.junit.FongoRule;
> > +
> > +import org.apache.calcite.util.Bug;
> > +import org.apache.calcite.util.Util;
> > +
> > +import com.google.common.base.Function;
> > +import com.google.common.base.Preconditions;
> > +import com.google.common.io.LineProcessor;
> > +import com.google.common.io.Resources;
> > import com.mongodb.client.MongoCollection;
> > import com.mongodb.client.MongoDatabase;
> >
> > +import net.hydromatic.foodmart.data.json.FoodmartJson;
> > +
> > +import org.bson.BsonDateTime;
> > import org.bson.BsonDocument;
> > +import org.bson.BsonInt32;
> > +import org.bson.BsonString;
> > import org.bson.Document;
> >
> > -import org.junit.Before;
> > +import org.hamcrest.CoreMatchers;
> > +
> > +import org.junit.Assert;
> > +import org.junit.BeforeClass;
> > +import org.junit.ClassRule;
> > import org.junit.Ignore;
> > -import org.junit.Rule;
> > import org.junit.Test;
> >
> > -import java.sql.Connection;
> > -import java.sql.DriverManager;
> > +import java.io.IOException;
> > +import java.io.UncheckedIOException;
> > +import java.net.URL;
> > +import java.nio.charset.StandardCharsets;
> > +import java.sql.ResultSet;
> > import java.sql.SQLException;
> > +import java.text.SimpleDateFormat;
> > +import java.util.Date;
> > +import java.util.List;
> > +import java.util.Locale;
> > +import java.util.Map;
> >
> > /**
> > - * Tests current adapter using in-memory (fake) implementation of Mongo
> API:
> > - * <a href="https://github.com/fakemongo/fongo">Fongo</a>.
> > - *
> > + * Testing mongo adapter functionality. By default runs with
> > + * <a href="https://github.com/fakemongo/fongo">Fongo</a> unless
> {@code IT} maven profile is enabled
> > + * (via {@code $ mvn -Pit install}).
> >  */
> > -public class MongoAdapterTest {
> > +public class MongoAdapterTest implements SchemaFactory {
> > +
> > +  /** Connection factory based on the "mongo-zips" model. */
> > +  protected static final URL MODEL =
> MongoAdapterTest.class.getResource("/mongo-model.json");
> > +
> > +  /** Number of records in local file */
> > +  protected static final int ZIPS_SIZE = 149;
> > +
> > +  @ClassRule
> > +  public static final MongoDatabaseRule RULE =
> MongoDatabaseRule.create();
> > +
> > +  private static MongoSchema schema;
> > +
> > +  @BeforeClass
> > +  public static void setUp() throws Exception {
> > +    MongoDatabase database = RULE.database();
> > +
> > +    populate(database.getCollection("zips"),
> MongoAdapterTest.class.getResource("/zips-mini.json"));
> > +    populate(database.getCollection("store"),
> FoodmartJson.class.getResource("/store.json"));
> > +    populate(database.getCollection("warehouse"),
> > +            FoodmartJson.class.getResource("/warehouse.json"));
> >
> > -  @Rule
> > -  public final FongoRule rule = new FongoRule();
> > +    // Manually insert data for data-time test.
> > +    MongoCollection<BsonDocument> datatypes =
> database.getCollection("datatypes")
> > +            .withDocumentClass(BsonDocument.class);
> > +    if (datatypes.count() > 0) {
> > +      datatypes.deleteMany(new BsonDocument());
> > +    }
> > +    BsonDocument doc = new BsonDocument();
> > +    Date date = new SimpleDateFormat("yyyy-MM-dd",
> Locale.getDefault()).parse("2012-09-05");
> > +    doc.put("date", new BsonDateTime(date.getTime()));
> > +    doc.put("value", new BsonInt32(1231));
> > +    doc.put("ownerId", new BsonString("531e7789e4b0853ddb861313"));
> > +    datatypes.insertOne(doc);
> >
> > -  private MongoDatabase mongoDb;
> > -  private MongoCollection<Document> zips;
> > +    schema = new MongoSchema(database);
> > +  }
> > +
> > +  private static void populate(MongoCollection<Document> collection,
> URL resource)
> > +          throws IOException {
> > +    Preconditions.checkNotNull(collection, "collection");
> > +
> > +    if (collection.count() > 0) {
> > +      // delete any existing documents (run from a clean set)
> > +      collection.deleteMany(new BsonDocument());
> > +    }
> >
> > -  @Before
> > -  public void setUp() throws Exception {
> > -    mongoDb = rule.getDatabase(getClass().getSimpleName());
> > -    zips = mongoDb.getCollection("zips");
> > +    MongoCollection<BsonDocument> bsonCollection =
> collection.withDocumentClass(BsonDocument.class);
> > +    Resources.readLines(resource, StandardCharsets.UTF_8, new
> LineProcessor<Void>() {
> > +      @Override public boolean processLine(String line) throws
> IOException {
> > +        bsonCollection.insertOne(BsonDocument.parse(line));
> > +        return true;
> > +      }
> > +
> > +      @Override public Void getResult() {
> > +        return null;
> > +      }
> > +    });
> >   }
> >
> >   /**
> > -   * Handcrafted connection where we manually added {@link MongoSchema}
> > +   *  Returns always the same schema to avoid initialization costs.
> >    */
> > -  private CalciteAssert.ConnectionFactory newConnectionFactory() {
> > -    return new CalciteAssert.ConnectionFactory() {
> > -      @Override public Connection createConnection() throws
> SQLException {
> > -        Connection connection =
> DriverManager.getConnection("jdbc:calcite:");
> > -        final SchemaPlus root =
> connection.unwrap(CalciteConnection.class).getRootSchema();
> > -        root.add("mongo", new MongoSchema(mongoDb));
> > -        return connection;
> > -      }
> > -    };
> > +  @Override public Schema create(SchemaPlus parentSchema, String name,
> > +                                 Map<String, Object> operand) {
> > +    return schema;
> > +  }
> > +
> > +  private CalciteAssert.AssertThat assertModel(String model) {
> > +    // ensure that Schema from this instance is being used
> > +    model = model.replace(MongoSchemaFactory.class.getName(),
> MongoAdapterTest.class.getName());
> > +
> > +    return CalciteAssert.that()
> > +            .withModel(model);
> > +  }
> > +
> > +  private CalciteAssert.AssertThat assertModel(URL url) {
> > +    Preconditions.checkNotNull(url, "url");
> > +    try {
> > +      return assertModel(Resources.toString(url,
> StandardCharsets.UTF_8));
> > +    } catch (IOException e) {
> > +      throw new UncheckedIOException(e);
> > +    }
> >   }
> >
> >   @Test
> > -  public void single() {
> > -    zips.insertOne(new Document());
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\"")
> > +  public void testSort() {
> > +    assertModel(MODEL)
> > +            .query("select * from zips order by state")
> > +            .returnsCount(ZIPS_SIZE)
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
> > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> > +
> > +  @Test public void testSortLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "order by state, id offset 2 rows fetch next 3
> rows only")
> > +            .returns("STATE=AK; ID=99801\n"
> > +                    + "STATE=AL; ID=35215\n"
> > +                    + "STATE=AL; ID=35401\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state', ID: '$_id'}}",
> > +                            "{$sort: {STATE: 1, ID: 1}}",
> > +                            "{$skip: 2}",
> > +                            "{$limit: 3}"));
> > +  }
> > +
> > +  @Test public void testOffsetLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "offset 2 fetch next 3 rows only")
> >             .runs()
> > -            .returnsCount(1);
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$skip: 2}",
> > +                            "{$limit: 3}",
> > +                            "{$project: {STATE: '$state', ID:
> '$_id'}}"));
> >   }
> >
> > -  @Test
> > -  public void empty() {
> > -    // for some reason fongo doesn't list collection if it was unused
> > -    zips.insertOne(new Document());
> > -    zips.deleteMany(new BsonDocument());
> > -
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\"")
> > +  @Test public void testLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "fetch next 3 rows only")
> >             .runs()
> > -            .returnsCount(0);
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$limit: 3}",
> > +                            "{$project: {STATE: '$state', ID:
> '$_id'}}"));
> >   }
> >
> > -  @Test
> > -  public void filter() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > +  @Ignore
> > +  @Test public void testFilterSort() {
> > +    // LONGITUDE and LATITUDE are null because of CALCITE-194.
> > +    Util.discard(Bug.CALCITE_194_FIXED);
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + "where city = 'SPRINGFIELD' and id >= '70000'\n"
> > +                    + "order by state, id")
> > +            .returns(""
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=752; STATE=AR; ID=72157\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=1992; STATE=CO; ID=81073\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=5597; STATE=LA; ID=70462\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=32384; STATE=OR; ID=97477\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=27521; STATE=OR; ID=97478\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  $match: {\n"
> > +                                    + "    city: \"SPRINGFIELD\",\n"
> > +                                    + "    _id: {\n"
> > +                                    + "      $gte: \"70000\"\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', LONGITUDE:
> '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
> > +                            "{$sort: {STATE: 1, ID: 1}}"))
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC],
> dir1=[ASC])\n"
> > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "      MongoFilter(condition=[AND(=(CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0,
> '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", '70000'))])\n"
> > +                    + "        MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] = 'NY'")
> > -            .returns("city=New York\n");
> > +  @Test public void testFilterSortDesc() {
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + "where pop BETWEEN 45000 AND 46000\n"
> > +                    + "order by state desc, pop")
> > +            .limit(4)
> > +            .returns("CITY=BECKLEY; LONGITUDE=null; LATITUDE=null;
> POP=45196; STATE=WV; ID=25801\n"
> > +                    + "CITY=ROCKERVILLE; LONGITUDE=null; LATITUDE=null;
> POP=45328; STATE=SD; ID=57701\n"
> > +                    + "CITY=PAWTUCKET; LONGITUDE=null; LATITUDE=null;
> POP=45442; STATE=RI; ID=02860\n"
> > +                    + "CITY=LAWTON; LONGITUDE=null; LATITUDE=null;
> POP=45542; STATE=OK; ID=73505\n");
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                   + " where _MAP['STATE'] = 'DC'")
> > -            .returns("city=Washington\n");
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testUnionPlan() {
> > +    assertModel(MODEL)
> > +            .query("select * from \"sales_fact_1997\"\n"
> > +                    + "union all\n"
> > +                    + "select * from \"sales_fact_1998\"")
> > +            .explainContains("PLAN=EnumerableUnion(all=[true])\n"
> > +                    + "  MongoToEnumerableConverter\n"
> > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> 'product_id')):DOUBLE])\n"
> > +                    + "      MongoTableScan(table=[[_foodmart,
> sales_fact_1997]])\n"
> > +                    + "  MongoToEnumerableConverter\n"
> > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> 'product_id')):DOUBLE])\n"
> > +                    + "      MongoTableScan(table=[[_foodmart,
> sales_fact_1998]])")
> > +            .limit(2)
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "product_id=337", "product_id=1512"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] in ('DC', 'NY')")
> > -            .returns("city=New York\ncity=Washington\n");
> > +  @Ignore(
> > +          "java.lang.ClassCastException: java.lang.Integer cannot be
> cast to java.lang.Double")
> > +  @Test public void testFilterUnionPlan() {
> > +    assertModel(MODEL)
> > +            .query("select * from (\n"
> > +                    + "  select * from \"sales_fact_1997\"\n"
> > +                    + "  union all\n"
> > +                    + "  select * from \"sales_fact_1998\")\n"
> > +                    + "where \"product_id\" = 1")
> > +            .runs();
> >   }
> >
> > -  @Test
> > -  public void limit() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > +  /** Tests that we don't generate multiple constraints on the same
> column.
> > +   * MongoDB doesn't like it. If there is an '=', it supersedes all
> other
> > +   * operators. */
> > +  @Test public void testFilterRedundant() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select * from zips where state > 'CA' and state <
> 'AZ' and state = 'OK'")
> > +            .runs()
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"state\": \"OK\"\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', LONGITUDE:
> '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID:
> '$_id'}}"));
> > +  }
> > +
> > +  @Test public void testSelectWhere() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select * from \"warehouse\" where
> \"warehouse_state_province\" = 'CA'")
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoProject(warehouse_id=[CAST(ITEM($0,
> 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0,
> 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> warehouse]])")
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "warehouse_id=6;
> warehouse_state_province=CA",
> > +                            "warehouse_id=7;
> warehouse_state_province=CA",
> > +                            "warehouse_id=14;
> warehouse_state_province=CA",
> > +                            "warehouse_id=24;
> warehouse_state_province=CA"))
> > +            .queryContains(
> > +                    // Per
> https://issues.apache.org/jira/browse/CALCITE-164,
> > +                    // $match must occur before $project for good
> performance.
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "
> \"warehouse_state_province\": \"CA\"\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {warehouse_id: 1,
> warehouse_state_province: 1}}"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\" limit 1")
> > -            .returnsCount(1);
> > +  @Test public void testInPlan() {
> > +    assertModel(MODEL)
> > +            .query("select \"store_id\", \"store_name\" from
> \"store\"\n"
> > +                    + "where \"store_name\" in ('Store 1', 'Store 10',
> 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "store_id=1; store_name=Store 1",
> > +                            "store_id=3; store_name=Store 3",
> > +                            "store_id=7; store_name=Store 7",
> > +                            "store_id=10; store_name=Store 10",
> > +                            "store_id=11; store_name=Store 11",
> > +                            "store_id=15; store_name=Store 15",
> > +                            "store_id=16; store_name=Store 16",
> > +                            "store_id=24; store_name=Store 24"))
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"$or\": [\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 1\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 10\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 11\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 15\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 16\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 24\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 3\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 7\"\n"
> > +                                    + "      }\n"
> > +                                    + "    ]\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {store_id: 1, store_name:
> 1}}"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\" limit 2")
> > -            .returnsCount(2);
> > +  /** Simple query based on the "mongo-zips" model. */
> > +  @Test public void testZips() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips")
> > +            .returnsCount(ZIPS_SIZE);
> > +  }
> >
> > +  @Test public void testCountGroupByEmpty() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) from zips")
> > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> ZIPS_SIZE))
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
> > +                    + "    MongoTableScan(table=[[mongo_raw, zips]])")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$group: {_id: {}, 'EXPR$0': {$sum:
> 1}}}"));
> >   }
> >
> > -  /**
> > -   * Following queries are not supported in Mongo adapter :
> > -   * <pre>
> > -   * {@code A and (B or C)}
> > -   * {@code (A or B) and C}
> > -   * </pre>
> > +  @Test public void testCountGroupByEmptyMultiplyBy2() {
> > +    // This operation is not supported by fongo:
> https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +
> > +    assertModel(MODEL)
> > +            .query("select count(*)*2 from zips")
> > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> ZIPS_SIZE * 2))
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$group: {_id: {}, _0: {$sum: 1}}}",
> > +                            "{$project: {'EXPR$0': {$multiply: ['$_0',
> {$literal: 2}]}}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumnNotProjected() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) from zips group by state order by
> 1")
> > +            .limit(2)
> > +            .returns("EXPR$0=2\n"
> > +                    + "EXPR$0=2\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', 'EXPR$0': {$sum:
> 1}}}",
> > +                            "{$project: {STATE: '$_id', 'EXPR$0':
> '$EXPR$0'}}",
> > +                            "{$project: {'EXPR$0': 1}}",
> > +                            "{$sort: {EXPR$0: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumn() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, count(*) as c from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; C=3\nSTATE=AL; C=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumnReversed() {
> > +    // Note extra $project compared to testGroupByOneColumn.
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select count(*) as c, state from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("C=3; STATE=AK\nC=3; STATE=AL\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{$project: {C: 1, STATE: 1}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByAvg() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, avg(pop) as a from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; A=26856\nSTATE=AL; A=43383\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', A: {$avg:
> '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', A: '$A'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByAvgSumCount() {
> > +    // This operation not supported by fongo:
> https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, avg(pop) as a, sum(pop) as s,
> count(pop) as c from zips group by state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; A=26856; S=80568; C=3\n"
> > +                    + "STATE=AL; A=43383; S=130151; C=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', _1: {$sum:
> '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', _1: '$_1', _2:
> '$_2'}}",
> > +                            "{$sort: {STATE: 1}}",
> > +                            "{$project: {STATE: 1, A: {$divide:
> [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S:
> {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
> > +  }
> > +
> > +  @Test public void testGroupByHaving() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(*) as c from zips\n"
> > +                    + "group by state having count(*) > 2 order by
> state")
> > +            .returnsCount(47)
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"C\": {\n"
> > +                                    + "      \"$gt\": 2\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
> > +  @Test public void testGroupByHaving2() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(*) as c from zips\n"
> > +                    + "group by state having sum(pop) > 12000000")
> > +            .returns("STATE=NY; C=1596\n"
> > +                    + "STATE=TX; C=1676\n"
> > +                    + "STATE=FL; C=826\n"
> > +                    + "STATE=CA; C=1523\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state', POP:
> '$pop'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}, _2:
> {$sum: '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C', _2:
> '$_2'}}",
> > +                            "{\n"
> > +                                    + "  $match: {\n"
> > +                                    + "    _2: {\n"
> > +                                    + "      $gt: 12000000\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {STATE: 1, C: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByMinMaxSum() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c, state,\n"
> > +                    + " min(pop) as min_pop, max(pop) as max_pop,
> sum(pop) as sum_pop\n"
> > +                    + "from zips group by state order by state")
> > +            .limit(2)
> > +            .returns("C=3; STATE=AK; MIN_POP=23238; MAX_POP=32383;
> SUM_POP=80568\n"
> > +                    + "C=3; STATE=AL; MIN_POP=42124; MAX_POP=44165;
> SUM_POP=130151\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1},
> MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum:
> '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C',
> MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
> > +                            "{$project: {C: 1, STATE: 1, MIN_POP: 1,
> MAX_POP: 1, SUM_POP: 1}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupComposite() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c, state, city from zips\n"
> > +                    + "group by state, city\n"
> > +                    + "order by c desc, city\n"
> > +                    + "limit 2")
> > +            .returns("C=1; STATE=SD; CITY=ABERDEEN\n"
> > +                      + "C=1; STATE=SC; CITY=AIKEN\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}, C: {$sum: 1}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE', C: '$C'}}",
> > +                            "{$sort: {C: -1, CITY: 1}}",
> > +                            "{$limit: 2}",
> > +                            "{$project: {C: 1, STATE: 1, CITY: 1}}"));
> > +  }
> > +
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testDistinctCount() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(distinct city) as cdc from
> zips\n"
> > +                    + "where state in ('CA', 'TX') group by state order
> by state")
> > +            .returns("STATE=CA; CDC=1072\n"
> > +                    + "STATE=TX; CDC=1233\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"$or\": [\n"
> > +                                    + "      {\n"
> > +                                    + "        \"state\": \"CA\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"state\": \"TX\"\n"
> > +                                    + "      }\n"
> > +                                    + "    ]\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE'}}",
> > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testDistinctCountOrderBy() {
> > +    // java.lang.ClassCastException: com.mongodb.BasicDBObject cannot
> be cast to java.lang.Number
> > +    // https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +    assertModel(MODEL)
> > +            .query("select state, count(distinct city) as cdc\n"
> > +                    + "from zips\n"
> > +                    + "group by state\n"
> > +                    + "order by cdc desc limit 5")
> > +            .returns("STATE=VA; CDC=3\n"
> > +                    + "STATE=NY; CDC=3\n"
> > +                    + "STATE=SC; CDC=3\n"
> > +                    + "STATE=RI; CDC=3\n"
> > +                    + "STATE=WV; CDC=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE'}}",
> > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> > +                            "{$sort: {CDC: -1}}",
> > +                            "{$limit: 5}"));
> > +  }
> > +
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testProject() {
> > +    assertModel(MODEL)
> > +            .query("select state, city, 0 as zero from zips order by
> state, city")
> > +            .limit(2)
> > +            .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
> > +                    + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$sort: {STATE: 1, CITY: 1}}",
> > +                            "{$project: {STATE: 1, CITY: 1, ZERO:
> {$literal: 0}}}"));
> > +  }
> > +
> > +  @Test public void testFilter() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where state = 'CA'")
> > +            .limit(2)
> > +            .returns("STATE=CA; CITY=LOS ANGELES\n"
> > +                    + "STATE=CA; CITY=BELL GARDENS\n")
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoProject(STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> > +
> > +  /** MongoDB's predicates are handed (they can only accept literals on
> the
> > +   * right-hand size) so it's worth testing that we handle them right
> both
> > +   * ways around. */
> > +  @Test public void testFilterReversed() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where 'WI' < state")
> > +            .limit(2)
> > +            .returns("STATE=WV; CITY=BECKLEY\nSTATE=WV; CITY=ELM
> GROVE\n");
> > +
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where state > 'WI'")
> > +            .limit(2)
> > +            .returns("STATE=WV; CITY=BECKLEY\n"
> > +                    + "STATE=WV; CITY=ELM GROVE\n");
> > +  }
> > +
> > +  /** MongoDB's predicates are handed (they can only accept literals on
> the
> > +   * right-hand size) so it's worth testing that we handle them right
> both
> > +   * ways around.
> >    *
> > -   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2331
> ">[CALCITE-2331]</a>
> > -   */
> > -  @Ignore("broken; [CALCITE-2331] is logged to fix it")
> > -  @Test
> > -  public void validateCALCITE2331() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > -
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] in ('DC', 'NY') and
> _MAP['CITY'] = 'New York'")
> > -            .returns("city=New York\n");
> > +   * <p>Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-740
> ">[CALCITE-740]
> > +   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>.
> */
> > +  @Test public void testFilterPair() {
> > +    final int gt9k = 148;
> > +    final int lt9k = 1;
> > +    final int gt8k = 148;
> > +    final int lt8k = 1;
> > +    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
> > +    checkPredicate(gt9k, "where pop > 9000");
> > +    checkPredicate(lt9k, "where pop < 9000");
> > +    checkPredicate(gt8k, "where pop > 8000");
> > +    checkPredicate(lt8k, "where pop < 8000");
> > +    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
> > +    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
> > +    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
> > +    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
> > +  }
> > +
> > +  private void checkPredicate(int expected, String q) {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c from zips\n"
> > +                    + q)
> > +            .returns("C=" + expected + "\n");
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + q)
> > +            .returnsCount(expected);
> > +  }
> > +
> > +  /** Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-286
> ">[CALCITE-286]
> > +   * Error casting MongoDB date</a>. */
> > +  @Test public void testDate() {
> > +    // Assumes that you have created the following collection before
> running
> > +    // this test:
> > +    //
> > +    // $ mongo
> > +    // > use test
> > +    // switched to db test
> > +    // > db.createCollection("datatypes")
> > +    // { "ok" : 1 }
> > +    // > db.datatypes.insert( {
> > +    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
> > +    //     "_class" : "com.ericblue.Test",
> > +    //     "date" : ISODate("2012-09-05T07:00:00Z"),
> > +    //     "value" : 1231,
> > +    //     "ownerId" : "531e7789e4b0853ddb861313"
> > +    //   } )
> > +    assertModel("{\n"
> > +            + "  version: '1.0',\n"
> > +            + "  defaultSchema: 'test',\n"
> > +            + "   schemas: [\n"
> > +            + "     {\n"
> > +            + "       type: 'custom',\n"
> > +            + "       name: 'test',\n"
> > +            + "       factory:
> 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> > +            + "       operand: {\n"
> > +            + "         host: 'localhost',\n"
> > +            + "         database: 'test'\n"
> > +            + "       }\n"
> > +            + "     }\n"
> > +            + "   ]\n"
> > +            + "}")
> > +            .query("select cast(_MAP['date'] as DATE) from
> \"datatypes\"")
> > +            .returnsUnordered("EXPR$0=2012-09-05");
> > +  }
> > +
> > +  /** Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-665
> ">[CALCITE-665]
> > +   * ClassCastException in MongoDB adapter</a>. */
> > +  @Test public void testCountViaInt() {
> > +    assertModel(MODEL)
> > +        .query("select count(*) from zips")
> > +        .returns(
> > +            new Function<ResultSet, Void>() {
> > +              public Void apply(ResultSet input) {
> > +                try {
> > +                  Assert.assertThat(input.next(),
> CoreMatchers.is(true));
> > +                  Assert.assertThat(input.getInt(1),
> CoreMatchers.is(ZIPS_SIZE));
> > +                  return null;
> > +                } catch (SQLException e) {
> > +                  throw new RuntimeException(e);
> > +                }
> > +              }
> > +            });
> > +  }
> > +
> > +  /** Returns a function that checks that a particular MongoDB pipeline
> is
> > +   * generated to implement a query. */
> > +  private static Function<List, Void> mongoChecker(final String...
> strings) {
> > +    return new Function<List, Void>() {
> > +      public Void apply(List actual) {
> > +        Object[] actualArray =
> > +                actual == null || actual.isEmpty()
> > +                        ? null
> > +                        : ((List) actual.get(0)).toArray();
> > +        CalciteAssert.assertArrayEqual("expected MongoDB query not
> found",
> > +                strings, actualArray);
> > +        return null;
> > +      }
> > +    };
> >   }
> > }
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > new file mode 100644
> > index 0000000..90bd759
> > --- /dev/null
> > +++
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > @@ -0,0 +1,83 @@
> > +/*
> > + * 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.calcite.adapter.mongodb;
> > +
> > +import org.apache.calcite.test.MongoAssertions;
> > +
> > +import com.github.fakemongo.Fongo;
> > +
> > +import com.google.common.base.Preconditions;
> > +
> > +import com.mongodb.MongoClient;
> > +import com.mongodb.client.MongoDatabase;
> > +
> > +import org.junit.rules.ExternalResource;
> > +
> > +/**
> > + * Instantiates new connection to fongo (or mongo) database depending
> on current profile
> > + * (unit or integration tests).
> > + *
> > + * By default, this rule is executed as part of a unit test and
> in-memory database
> > + * <a href="https://github.com/fakemongo/fongo">fongo</a> is used.
> > + *
> > + * <p>However, if maven profile is set to {@code IT} (eg. via command
> line
> > + * {@code $ mvn -Pit install}) this rule will connect to existing
> (external)
> > + * mongo instance ({@code localhost})</p>
> > + *
> > + */
> > +class MongoDatabaseRule extends ExternalResource {
> > +
> > +  private static final String DB_NAME = "test";
> > +
> > +  private final MongoDatabase database;
> > +  private final MongoClient client;
> > +
> > +  private MongoDatabaseRule(MongoClient client) {
> > +    this.client = Preconditions.checkNotNull(client, "client");
> > +    this.database = client.getDatabase(DB_NAME);
> > +  }
> > +
> > +  /**
> > +   * Create an instance based on current maven profile (as defined by
> {@code -Pit}).
> > +   */
> > +  static MongoDatabaseRule create() {
> > +    final MongoClient client;
> > +    if (MongoAssertions.useMongo()) {
> > +      // use to real client (connects to mongo)
> > +      client = new MongoClient();
> > +    } else if (MongoAssertions.useFongo()) {
> > +      // in-memory DB (fake Mongo)
> > +      client = new
> Fongo(MongoDatabaseRule.class.getSimpleName()).getMongo();
> > +    } else {
> > +      throw new UnsupportedOperationException("I can only connect to
> Mongo or Fongo instances");
> > +    }
> > +
> > +    return new MongoDatabaseRule(client);
> > +  }
> > +
> > +
> > +  MongoDatabase database() {
> > +    return database;
> > +  }
> > +
> > +  @Override protected void after() {
> > +    client.close();
> > +  }
> > +
> > +}
> > +
> > +// End MongoDatabaseRule.java
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > index 00ed03a..bda1163 100644
> > --- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > +++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > @@ -16,829 +16,29 @@
> >  */
> > package org.apache.calcite.test;
> >
> > -import org.apache.calcite.linq4j.Ord;
> > -import org.apache.calcite.util.Bug;
> > -import org.apache.calcite.util.Pair;
> > -import org.apache.calcite.util.Util;
> > +import org.apache.calcite.adapter.mongodb.MongoAdapterTest;
> >
> > -import com.google.common.base.Function;
> > -import com.google.common.collect.ImmutableMap;
> > -import com.google.common.collect.Lists;
> > -import com.google.common.collect.Ordering;
> > +import org.junit.BeforeClass;
> >
> > -import org.hamcrest.CoreMatchers;
> > -import org.junit.Ignore;
> > -import org.junit.Test;
> > -
> > -import java.sql.ResultSet;
> > -import java.sql.SQLException;
> > -import java.util.Arrays;
> > -import java.util.Collections;
> > -import java.util.List;
> > -
> > -import static org.hamcrest.CoreMatchers.equalTo;
> > -import static org.junit.Assert.assertThat;
> > +import static org.junit.Assume.assumeTrue;
> >
> > /**
> > - * Tests for the {@code org.apache.calcite.adapter.mongodb} package.
> > - *
> > - * <p>Before calling this test, you need to populate MongoDB, as
> follows:
> > + * Used to trigger integration tests from maven (thus class name is
> suffixed with {@code IT}).
> >  *
> > - * <blockquote><code>
> > - * git clone https://github.com/vlsi/calcite-test-dataset<br>
> > - * cd calcite-test-dataset<br>
> > - * mvn install
> > - * </code></blockquote>
> > + * <p>If you want to run integration tests from IDE manually set
> > + * {@code -Dcalcite.integrationTest=true} system property.
> >  *
> > - * <p>This will create a virtual machine with MongoDB and "zips" and
> "foodmart"
> > - * data sets.
> > + * For command line use:
> > + * <pre>
> > + *     $ mvn install -Pit
> > + * </pre>
> > + * </p>
> >  */
> > -public class MongoAdapterIT {
> > -  public static final String MONGO_FOODMART_SCHEMA = "     {\n"
> > -      + "       type: 'custom',\n"
> > -      + "       name: '_foodmart',\n"
> > -      + "       factory:
> 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> > -      + "       operand: {\n"
> > -      + "         host: 'localhost',\n"
> > -      + "         database: 'foodmart'\n"
> > -      + "       }\n"
> > -      + "     },\n"
> > -      + "     {\n"
> > -      + "       name: 'foodmart',\n"
> > -      + "       tables: [\n"
> > -      + "         {\n"
> > -      + "           name: 'sales_fact_1997',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double)
> AS \"product_id\" from \"_foodmart\".\"sales_fact_1997\"'\n"
> > -      + "         },\n"
> > -      + "         {\n"
> > -      + "           name: 'sales_fact_1998',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double)
> AS \"product_id\" from \"_foodmart\".\"sales_fact_1998\"'\n"
> > -      + "         },\n"
> > -      + "         {\n"
> > -      + "           name: 'store',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'store_id\\'] AS double)
> AS \"store_id\", cast(_MAP[\\'store_name\\'] AS varchar(20)) AS
> \"store_name\" from \"_foodmart\".\"store\"'\n"
> > -      + "         },\n"
> > -      + "

Re: [2/2] calcite git commit: [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

Posted by Andrei Sereda <an...@sereda.cc>.
Sorry it was my commit. I should have run site (javadoc) task before
submitting PR.

On Thu, Jun 14, 2018 at 11:05 AM Michael Mior <mm...@apache.org> wrote:

> Done. Thanks for consistently running those tests.
> --
> Michael Mior
> mmior@apache.org
>
>
> Le jeu. 14 juin 2018 à 10:42, Julian Hyde <jh...@apache.org> a écrit :
>
> > This change introduces a javadoc error:
> >
> > [ERROR]
> >
> /home/jhyde/regress/calcite/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java:35:
> > error: unexpected end tag: </p>
> > [ERROR]  * </p>
> >
> > Please fix ASAP.
> >
> >
> > > On Jun 13, 2018, at 6:00 PM, mmior@apache.org wrote:
> > >
> > > [CALCITE-2345] Running Unit tests with Fongo and integration tests with
> > real mongo instance (Andrei Sereda)
> > >
> > > Better test coverage for unit tests using Fongo in-memory
> implementation
> > of Mongo API.
> > > New code will decide (at runtime) what connection to make: fongo vs
> > mongo. Identical tests will be run against
> > > both databases (depending on maven profile surefire vs failsafe)
> > >
> > > Close apache/calcite#723
> > >
> > >
> > > Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
> > > Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/dcf396a5
> > > Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/dcf396a5
> > > Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/dcf396a5
> > >
> > > Branch: refs/heads/master
> > > Commit: dcf396a5ca92ee0714c28d9a3c310f53f46220ab
> > > Parents: f84a3eb
> > > Author: Andrei Sereda <an...@nospam.com>
> > > Authored: Tue Jun 5 16:23:56 2018 -0400
> > > Committer: Michael Mior <mm...@uwaterloo.ca>
> > > Committed: Wed Jun 13 18:59:15 2018 -0400
> > >
> > > ----------------------------------------------------------------------
> > > mongodb/pom.xml                                 |  10 +-
> > > .../adapter/mongodb/MongoAdapterTest.java       | 794
> +++++++++++++++---
> > > .../adapter/mongodb/MongoDatabaseRule.java      |  83 ++
> > > .../org/apache/calcite/test/MongoAdapterIT.java | 830
> +------------------
> > > .../apache/calcite/test/MongoAssertions.java    | 101 +++
> > > .../test/resources/mongo-foodmart-model.json    | 221 -----
> > > mongodb/src/test/resources/mongo-model.json     |  70 ++
> > > .../src/test/resources/mongo-zips-model.json    |  41 -
> > > mongodb/src/test/resources/zips-mini.json       | 149 ++++
> > > pom.xml                                         |   9 +
> > > 10 files changed, 1132 insertions(+), 1176 deletions(-)
> > > ----------------------------------------------------------------------
> > >
> > >
> > >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/pom.xml
> > > ----------------------------------------------------------------------
> > > diff --git a/mongodb/pom.xml b/mongodb/pom.xml
> > > index 25c0642..63cc147 100644
> > > --- a/mongodb/pom.xml
> > > +++ b/mongodb/pom.xml
> > > @@ -71,6 +71,11 @@ limitations under the License.
> > >       <scope>test</scope>
> > >     </dependency>
> > >     <dependency>
> > > +      <groupId>net.hydromatic</groupId>
> > > +      <artifactId>foodmart-data-json</artifactId>
> > > +      <scope>test</scope>
> > > +    </dependency>
> > > +    <dependency>
> > >       <groupId>org.mongodb</groupId>
> > >       <artifactId>mongo-java-driver</artifactId>
> > >     </dependency>
> > > @@ -78,11 +83,6 @@ limitations under the License.
> > >       <groupId>org.slf4j</groupId>
> > >       <artifactId>slf4j-api</artifactId>
> > >     </dependency>
> > > -    <dependency>
> > > -      <groupId>org.slf4j</groupId>
> > > -      <artifactId>slf4j-log4j12</artifactId>
> > > -      <scope>test</scope>
> > > -    </dependency>
> > >   </dependencies>
> > >
> > >   <build>
> > >
> > >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > > ----------------------------------------------------------------------
> > > diff --git
> >
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> >
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > > index b6ca4f3..a4061e4 100644
> > > ---
> >
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > > +++
> >
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > > @@ -16,143 +16,749 @@
> > >  */
> > > package org.apache.calcite.adapter.mongodb;
> > >
> > > -import org.apache.calcite.jdbc.CalciteConnection;
> > > +import org.apache.calcite.schema.Schema;
> > > +import org.apache.calcite.schema.SchemaFactory;
> > > import org.apache.calcite.schema.SchemaPlus;
> > > import org.apache.calcite.test.CalciteAssert;
> > > +import org.apache.calcite.test.MongoAssertions;
> > >
> > > -import com.github.fakemongo.junit.FongoRule;
> > > +
> > > +import org.apache.calcite.util.Bug;
> > > +import org.apache.calcite.util.Util;
> > > +
> > > +import com.google.common.base.Function;
> > > +import com.google.common.base.Preconditions;
> > > +import com.google.common.io.LineProcessor;
> > > +import com.google.common.io.Resources;
> > > import com.mongodb.client.MongoCollection;
> > > import com.mongodb.client.MongoDatabase;
> > >
> > > +import net.hydromatic.foodmart.data.json.FoodmartJson;
> > > +
> > > +import org.bson.BsonDateTime;
> > > import org.bson.BsonDocument;
> > > +import org.bson.BsonInt32;
> > > +import org.bson.BsonString;
> > > import org.bson.Document;
> > >
> > > -import org.junit.Before;
> > > +import org.hamcrest.CoreMatchers;
> > > +
> > > +import org.junit.Assert;
> > > +import org.junit.BeforeClass;
> > > +import org.junit.ClassRule;
> > > import org.junit.Ignore;
> > > -import org.junit.Rule;
> > > import org.junit.Test;
> > >
> > > -import java.sql.Connection;
> > > -import java.sql.DriverManager;
> > > +import java.io.IOException;
> > > +import java.io.UncheckedIOException;
> > > +import java.net.URL;
> > > +import java.nio.charset.StandardCharsets;
> > > +import java.sql.ResultSet;
> > > import java.sql.SQLException;
> > > +import java.text.SimpleDateFormat;
> > > +import java.util.Date;
> > > +import java.util.List;
> > > +import java.util.Locale;
> > > +import java.util.Map;
> > >
> > > /**
> > > - * Tests current adapter using in-memory (fake) implementation of
> Mongo
> > API:
> > > - * <a href="https://github.com/fakemongo/fongo">Fongo</a>.
> > > - *
> > > + * Testing mongo adapter functionality. By default runs with
> > > + * <a href="https://github.com/fakemongo/fongo">Fongo</a> unless
> > {@code IT} maven profile is enabled
> > > + * (via {@code $ mvn -Pit install}).
> > >  */
> > > -public class MongoAdapterTest {
> > > +public class MongoAdapterTest implements SchemaFactory {
> > > +
> > > +  /** Connection factory based on the "mongo-zips" model. */
> > > +  protected static final URL MODEL =
> > MongoAdapterTest.class.getResource("/mongo-model.json");
> > > +
> > > +  /** Number of records in local file */
> > > +  protected static final int ZIPS_SIZE = 149;
> > > +
> > > +  @ClassRule
> > > +  public static final MongoDatabaseRule RULE =
> > MongoDatabaseRule.create();
> > > +
> > > +  private static MongoSchema schema;
> > > +
> > > +  @BeforeClass
> > > +  public static void setUp() throws Exception {
> > > +    MongoDatabase database = RULE.database();
> > > +
> > > +    populate(database.getCollection("zips"),
> > MongoAdapterTest.class.getResource("/zips-mini.json"));
> > > +    populate(database.getCollection("store"),
> > FoodmartJson.class.getResource("/store.json"));
> > > +    populate(database.getCollection("warehouse"),
> > > +            FoodmartJson.class.getResource("/warehouse.json"));
> > >
> > > -  @Rule
> > > -  public final FongoRule rule = new FongoRule();
> > > +    // Manually insert data for data-time test.
> > > +    MongoCollection<BsonDocument> datatypes =
> > database.getCollection("datatypes")
> > > +            .withDocumentClass(BsonDocument.class);
> > > +    if (datatypes.count() > 0) {
> > > +      datatypes.deleteMany(new BsonDocument());
> > > +    }
> > > +    BsonDocument doc = new BsonDocument();
> > > +    Date date = new SimpleDateFormat("yyyy-MM-dd",
> > Locale.getDefault()).parse("2012-09-05");
> > > +    doc.put("date", new BsonDateTime(date.getTime()));
> > > +    doc.put("value", new BsonInt32(1231));
> > > +    doc.put("ownerId", new BsonString("531e7789e4b0853ddb861313"));
> > > +    datatypes.insertOne(doc);
> > >
> > > -  private MongoDatabase mongoDb;
> > > -  private MongoCollection<Document> zips;
> > > +    schema = new MongoSchema(database);
> > > +  }
> > > +
> > > +  private static void populate(MongoCollection<Document> collection,
> > URL resource)
> > > +          throws IOException {
> > > +    Preconditions.checkNotNull(collection, "collection");
> > > +
> > > +    if (collection.count() > 0) {
> > > +      // delete any existing documents (run from a clean set)
> > > +      collection.deleteMany(new BsonDocument());
> > > +    }
> > >
> > > -  @Before
> > > -  public void setUp() throws Exception {
> > > -    mongoDb = rule.getDatabase(getClass().getSimpleName());
> > > -    zips = mongoDb.getCollection("zips");
> > > +    MongoCollection<BsonDocument> bsonCollection =
> > collection.withDocumentClass(BsonDocument.class);
> > > +    Resources.readLines(resource, StandardCharsets.UTF_8, new
> > LineProcessor<Void>() {
> > > +      @Override public boolean processLine(String line) throws
> > IOException {
> > > +        bsonCollection.insertOne(BsonDocument.parse(line));
> > > +        return true;
> > > +      }
> > > +
> > > +      @Override public Void getResult() {
> > > +        return null;
> > > +      }
> > > +    });
> > >   }
> > >
> > >   /**
> > > -   * Handcrafted connection where we manually added {@link
> MongoSchema}
> > > +   *  Returns always the same schema to avoid initialization costs.
> > >    */
> > > -  private CalciteAssert.ConnectionFactory newConnectionFactory() {
> > > -    return new CalciteAssert.ConnectionFactory() {
> > > -      @Override public Connection createConnection() throws
> > SQLException {
> > > -        Connection connection =
> > DriverManager.getConnection("jdbc:calcite:");
> > > -        final SchemaPlus root =
> > connection.unwrap(CalciteConnection.class).getRootSchema();
> > > -        root.add("mongo", new MongoSchema(mongoDb));
> > > -        return connection;
> > > -      }
> > > -    };
> > > +  @Override public Schema create(SchemaPlus parentSchema, String name,
> > > +                                 Map<String, Object> operand) {
> > > +    return schema;
> > > +  }
> > > +
> > > +  private CalciteAssert.AssertThat assertModel(String model) {
> > > +    // ensure that Schema from this instance is being used
> > > +    model = model.replace(MongoSchemaFactory.class.getName(),
> > MongoAdapterTest.class.getName());
> > > +
> > > +    return CalciteAssert.that()
> > > +            .withModel(model);
> > > +  }
> > > +
> > > +  private CalciteAssert.AssertThat assertModel(URL url) {
> > > +    Preconditions.checkNotNull(url, "url");
> > > +    try {
> > > +      return assertModel(Resources.toString(url,
> > StandardCharsets.UTF_8));
> > > +    } catch (IOException e) {
> > > +      throw new UncheckedIOException(e);
> > > +    }
> > >   }
> > >
> > >   @Test
> > > -  public void single() {
> > > -    zips.insertOne(new Document());
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select * from \"mongo\".\"zips\"")
> > > +  public void testSort() {
> > > +    assertModel(MODEL)
> > > +            .query("select * from zips order by state")
> > > +            .returnsCount(ZIPS_SIZE)
> > > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > > +                    + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
> > > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> > 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> > 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> > POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> > 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> > CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > > +                    + "      MongoTableScan(table=[[mongo_raw,
> > zips]])");
> > > +  }
> > > +
> > > +  @Test public void testSortLimit() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, id from zips\n"
> > > +                    + "order by state, id offset 2 rows fetch next 3
> > rows only")
> > > +            .returns("STATE=AK; ID=99801\n"
> > > +                    + "STATE=AL; ID=35215\n"
> > > +                    + "STATE=AL; ID=35401\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state', ID:
> '$_id'}}",
> > > +                            "{$sort: {STATE: 1, ID: 1}}",
> > > +                            "{$skip: 2}",
> > > +                            "{$limit: 3}"));
> > > +  }
> > > +
> > > +  @Test public void testOffsetLimit() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, id from zips\n"
> > > +                    + "offset 2 fetch next 3 rows only")
> > >             .runs()
> > > -            .returnsCount(1);
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$skip: 2}",
> > > +                            "{$limit: 3}",
> > > +                            "{$project: {STATE: '$state', ID:
> > '$_id'}}"));
> > >   }
> > >
> > > -  @Test
> > > -  public void empty() {
> > > -    // for some reason fongo doesn't list collection if it was unused
> > > -    zips.insertOne(new Document());
> > > -    zips.deleteMany(new BsonDocument());
> > > -
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select * from \"mongo\".\"zips\"")
> > > +  @Test public void testLimit() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, id from zips\n"
> > > +                    + "fetch next 3 rows only")
> > >             .runs()
> > > -            .returnsCount(0);
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$limit: 3}",
> > > +                            "{$project: {STATE: '$state', ID:
> > '$_id'}}"));
> > >   }
> > >
> > > -  @Test
> > > -  public void filter() {
> > > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> > "NY"));
> > > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> > "DC"));
> > > +  @Ignore
> > > +  @Test public void testFilterSort() {
> > > +    // LONGITUDE and LATITUDE are null because of CALCITE-194.
> > > +    Util.discard(Bug.CALCITE_194_FIXED);
> > > +    assertModel(MODEL)
> > > +            .query("select * from zips\n"
> > > +                    + "where city = 'SPRINGFIELD' and id >= '70000'\n"
> > > +                    + "order by state, id")
> > > +            .returns(""
> > > +                    + "CITY=SPRINGFIELD; LONGITUDE=null;
> LATITUDE=null;
> > POP=752; STATE=AR; ID=72157\n"
> > > +                    + "CITY=SPRINGFIELD; LONGITUDE=null;
> LATITUDE=null;
> > POP=1992; STATE=CO; ID=81073\n"
> > > +                    + "CITY=SPRINGFIELD; LONGITUDE=null;
> LATITUDE=null;
> > POP=5597; STATE=LA; ID=70462\n"
> > > +                    + "CITY=SPRINGFIELD; LONGITUDE=null;
> LATITUDE=null;
> > POP=32384; STATE=OR; ID=97477\n"
> > > +                    + "CITY=SPRINGFIELD; LONGITUDE=null;
> LATITUDE=null;
> > POP=27521; STATE=OR; ID=97478\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{\n"
> > > +                                    + "  $match: {\n"
> > > +                                    + "    city: \"SPRINGFIELD\",\n"
> > > +                                    + "    _id: {\n"
> > > +                                    + "      $gte: \"70000\"\n"
> > > +                                    + "    }\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {CITY: '$city', LONGITUDE:
> > '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID:
> '$_id'}}",
> > > +                            "{$sort: {STATE: 1, ID: 1}}"))
> > > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > > +                    + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC],
> > dir1=[ASC])\n"
> > > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> > 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> > 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> > POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> > 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> > CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > > +                    + "
> MongoFilter(condition=[AND(=(CAST(ITEM($0,
> > 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0,
> > '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\", '70000'))])\n"
> > > +                    + "        MongoTableScan(table=[[mongo_raw,
> > zips]])");
> > > +  }
> > >
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> > \"city\" from \"mongo\".\"zips\" "
> > > -                    + " where _MAP['STATE'] = 'NY'")
> > > -            .returns("city=New York\n");
> > > +  @Test public void testFilterSortDesc() {
> > > +    assertModel(MODEL)
> > > +            .query("select * from zips\n"
> > > +                    + "where pop BETWEEN 45000 AND 46000\n"
> > > +                    + "order by state desc, pop")
> > > +            .limit(4)
> > > +            .returns("CITY=BECKLEY; LONGITUDE=null; LATITUDE=null;
> > POP=45196; STATE=WV; ID=25801\n"
> > > +                    + "CITY=ROCKERVILLE; LONGITUDE=null;
> LATITUDE=null;
> > POP=45328; STATE=SD; ID=57701\n"
> > > +                    + "CITY=PAWTUCKET; LONGITUDE=null; LATITUDE=null;
> > POP=45442; STATE=RI; ID=02860\n"
> > > +                    + "CITY=LAWTON; LONGITUDE=null; LATITUDE=null;
> > POP=45542; STATE=OK; ID=73505\n");
> > > +  }
> > >
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> > \"city\" from \"mongo\".\"zips\" "
> > > -                   + " where _MAP['STATE'] = 'DC'")
> > > -            .returns("city=Washington\n");
> > > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > > +  @Test public void testUnionPlan() {
> > > +    assertModel(MODEL)
> > > +            .query("select * from \"sales_fact_1997\"\n"
> > > +                    + "union all\n"
> > > +                    + "select * from \"sales_fact_1998\"")
> > > +            .explainContains("PLAN=EnumerableUnion(all=[true])\n"
> > > +                    + "  MongoToEnumerableConverter\n"
> > > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> > 'product_id')):DOUBLE])\n"
> > > +                    + "      MongoTableScan(table=[[_foodmart,
> > sales_fact_1997]])\n"
> > > +                    + "  MongoToEnumerableConverter\n"
> > > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> > 'product_id')):DOUBLE])\n"
> > > +                    + "      MongoTableScan(table=[[_foodmart,
> > sales_fact_1998]])")
> > > +            .limit(2)
> > > +            .returns(
> > > +                    MongoAssertions.checkResultUnordered(
> > > +                            "product_id=337", "product_id=1512"));
> > > +  }
> > >
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> > \"city\" from \"mongo\".\"zips\" "
> > > -                    + " where _MAP['STATE'] in ('DC', 'NY')")
> > > -            .returns("city=New York\ncity=Washington\n");
> > > +  @Ignore(
> > > +          "java.lang.ClassCastException: java.lang.Integer cannot be
> > cast to java.lang.Double")
> > > +  @Test public void testFilterUnionPlan() {
> > > +    assertModel(MODEL)
> > > +            .query("select * from (\n"
> > > +                    + "  select * from \"sales_fact_1997\"\n"
> > > +                    + "  union all\n"
> > > +                    + "  select * from \"sales_fact_1998\")\n"
> > > +                    + "where \"product_id\" = 1")
> > > +            .runs();
> > >   }
> > >
> > > -  @Test
> > > -  public void limit() {
> > > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> > "NY"));
> > > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> > "DC"));
> > > +  /** Tests that we don't generate multiple constraints on the same
> > column.
> > > +   * MongoDB doesn't like it. If there is an '=', it supersedes all
> > other
> > > +   * operators. */
> > > +  @Test public void testFilterRedundant() {
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select * from zips where state > 'CA' and state <
> > 'AZ' and state = 'OK'")
> > > +            .runs()
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{\n"
> > > +                                    + "  \"$match\": {\n"
> > > +                                    + "    \"state\": \"OK\"\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {CITY: '$city', LONGITUDE:
> > '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID:
> > '$_id'}}"));
> > > +  }
> > > +
> > > +  @Test public void testSelectWhere() {
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select * from \"warehouse\" where
> > \"warehouse_state_province\" = 'CA'")
> > > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > > +                    + "  MongoProject(warehouse_id=[CAST(ITEM($0,
> > 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0,
> > 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> > COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> > 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> > COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > > +                    + "      MongoTableScan(table=[[mongo_raw,
> > warehouse]])")
> > > +            .returns(
> > > +                    MongoAssertions.checkResultUnordered(
> > > +                            "warehouse_id=6;
> > warehouse_state_province=CA",
> > > +                            "warehouse_id=7;
> > warehouse_state_province=CA",
> > > +                            "warehouse_id=14;
> > warehouse_state_province=CA",
> > > +                            "warehouse_id=24;
> > warehouse_state_province=CA"))
> > > +            .queryContains(
> > > +                    // Per
> > https://issues.apache.org/jira/browse/CALCITE-164,
> > > +                    // $match must occur before $project for good
> > performance.
> > > +                    mongoChecker(
> > > +                            "{\n"
> > > +                                    + "  \"$match\": {\n"
> > > +                                    + "
> > \"warehouse_state_province\": \"CA\"\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {warehouse_id: 1,
> > warehouse_state_province: 1}}"));
> > > +  }
> > >
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select * from \"mongo\".\"zips\" limit 1")
> > > -            .returnsCount(1);
> > > +  @Test public void testInPlan() {
> > > +    assertModel(MODEL)
> > > +            .query("select \"store_id\", \"store_name\" from
> > \"store\"\n"
> > > +                    + "where \"store_name\" in ('Store 1', 'Store 10',
> > 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
> > > +            .returns(
> > > +                    MongoAssertions.checkResultUnordered(
> > > +                            "store_id=1; store_name=Store 1",
> > > +                            "store_id=3; store_name=Store 3",
> > > +                            "store_id=7; store_name=Store 7",
> > > +                            "store_id=10; store_name=Store 10",
> > > +                            "store_id=11; store_name=Store 11",
> > > +                            "store_id=15; store_name=Store 15",
> > > +                            "store_id=16; store_name=Store 16",
> > > +                            "store_id=24; store_name=Store 24"))
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{\n"
> > > +                                    + "  \"$match\": {\n"
> > > +                                    + "    \"$or\": [\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 1\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 10\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 11\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 15\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 16\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 24\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 3\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"store_name\": \"Store
> > 7\"\n"
> > > +                                    + "      }\n"
> > > +                                    + "    ]\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {store_id: 1, store_name:
> > 1}}"));
> > > +  }
> > >
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select * from \"mongo\".\"zips\" limit 2")
> > > -            .returnsCount(2);
> > > +  /** Simple query based on the "mongo-zips" model. */
> > > +  @Test public void testZips() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, city from zips")
> > > +            .returnsCount(ZIPS_SIZE);
> > > +  }
> > >
> > > +  @Test public void testCountGroupByEmpty() {
> > > +    assertModel(MODEL)
> > > +            .query("select count(*) from zips")
> > > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> > ZIPS_SIZE))
> > > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > > +                    + "  MongoAggregate(group=[{}],
> EXPR$0=[COUNT()])\n"
> > > +                    + "    MongoTableScan(table=[[mongo_raw, zips]])")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$group: {_id: {}, 'EXPR$0': {$sum:
> > 1}}}"));
> > >   }
> > >
> > > -  /**
> > > -   * Following queries are not supported in Mongo adapter :
> > > -   * <pre>
> > > -   * {@code A and (B or C)}
> > > -   * {@code (A or B) and C}
> > > -   * </pre>
> > > +  @Test public void testCountGroupByEmptyMultiplyBy2() {
> > > +    // This operation is not supported by fongo:
> > https://github.com/fakemongo/fongo/issues/152
> > > +    MongoAssertions.assumeRealMongoInstance();
> > > +
> > > +    assertModel(MODEL)
> > > +            .query("select count(*)*2 from zips")
> > > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> > ZIPS_SIZE * 2))
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$group: {_id: {}, _0: {$sum: 1}}}",
> > > +                            "{$project: {'EXPR$0': {$multiply: ['$_0',
> > {$literal: 2}]}}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByOneColumnNotProjected() {
> > > +    assertModel(MODEL)
> > > +            .query("select count(*) from zips group by state order by
> > 1")
> > > +            .limit(2)
> > > +            .returns("EXPR$0=2\n"
> > > +                    + "EXPR$0=2\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state'}}",
> > > +                            "{$group: {_id: '$STATE', 'EXPR$0': {$sum:
> > 1}}}",
> > > +                            "{$project: {STATE: '$_id', 'EXPR$0':
> > '$EXPR$0'}}",
> > > +                            "{$project: {'EXPR$0': 1}}",
> > > +                            "{$sort: {EXPR$0: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByOneColumn() {
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select state, count(*) as c from zips group by
> > state order by state")
> > > +            .limit(2)
> > > +            .returns("STATE=AK; C=3\nSTATE=AL; C=3\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state'}}",
> > > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByOneColumnReversed() {
> > > +    // Note extra $project compared to testGroupByOneColumn.
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select count(*) as c, state from zips group by
> > state order by state")
> > > +            .limit(2)
> > > +            .returns("C=3; STATE=AK\nC=3; STATE=AL\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state'}}",
> > > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > > +                            "{$project: {C: 1, STATE: 1}}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByAvg() {
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select state, avg(pop) as a from zips group by
> > state order by state")
> > > +            .limit(2)
> > > +            .returns("STATE=AK; A=26856\nSTATE=AL; A=43383\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {POP: '$pop', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: '$STATE', A: {$avg:
> > '$POP'}}}",
> > > +                            "{$project: {STATE: '$_id', A: '$A'}}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByAvgSumCount() {
> > > +    // This operation not supported by fongo:
> > https://github.com/fakemongo/fongo/issues/152
> > > +    MongoAssertions.assumeRealMongoInstance();
> > > +    assertModel(MODEL)
> > > +            .query(
> > > +                    "select state, avg(pop) as a, sum(pop) as s,
> > count(pop) as c from zips group by state order by state")
> > > +            .limit(2)
> > > +            .returns("STATE=AK; A=26856; S=80568; C=3\n"
> > > +                    + "STATE=AL; A=43383; S=130151; C=3\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {POP: '$pop', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: '$STATE', _1: {$sum:
> > '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
> > > +                            "{$project: {STATE: '$_id', _1: '$_1', _2:
> > '$_2'}}",
> > > +                            "{$sort: {STATE: 1}}",
> > > +                            "{$project: {STATE: 1, A: {$divide:
> > [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S:
> > {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByHaving() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, count(*) as c from zips\n"
> > > +                    + "group by state having count(*) > 2 order by
> > state")
> > > +            .returnsCount(47)
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state'}}",
> > > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > > +                            "{\n"
> > > +                                    + "  \"$match\": {\n"
> > > +                                    + "    \"C\": {\n"
> > > +                                    + "      \"$gt\": 2\n"
> > > +                                    + "    }\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
> > > +  @Test public void testGroupByHaving2() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, count(*) as c from zips\n"
> > > +                    + "group by state having sum(pop) > 12000000")
> > > +            .returns("STATE=NY; C=1596\n"
> > > +                    + "STATE=TX; C=1676\n"
> > > +                    + "STATE=FL; C=826\n"
> > > +                    + "STATE=CA; C=1523\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {STATE: '$state', POP:
> > '$pop'}}",
> > > +                            "{$group: {_id: '$STATE', C: {$sum: 1},
> _2:
> > {$sum: '$POP'}}}",
> > > +                            "{$project: {STATE: '$_id', C: '$C', _2:
> > '$_2'}}",
> > > +                            "{\n"
> > > +                                    + "  $match: {\n"
> > > +                                    + "    _2: {\n"
> > > +                                    + "      $gt: 12000000\n"
> > > +                                    + "    }\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {STATE: 1, C: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupByMinMaxSum() {
> > > +    assertModel(MODEL)
> > > +            .query("select count(*) as c, state,\n"
> > > +                    + " min(pop) as min_pop, max(pop) as max_pop,
> > sum(pop) as sum_pop\n"
> > > +                    + "from zips group by state order by state")
> > > +            .limit(2)
> > > +            .returns("C=3; STATE=AK; MIN_POP=23238; MAX_POP=32383;
> > SUM_POP=80568\n"
> > > +                    + "C=3; STATE=AL; MIN_POP=42124; MAX_POP=44165;
> > SUM_POP=130151\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {POP: '$pop', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: '$STATE', C: {$sum: 1},
> > MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum:
> > '$POP'}}}",
> > > +                            "{$project: {STATE: '$_id', C: '$C',
> > MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
> > > +                            "{$project: {C: 1, STATE: 1, MIN_POP: 1,
> > MAX_POP: 1, SUM_POP: 1}}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testGroupComposite() {
> > > +    assertModel(MODEL)
> > > +            .query("select count(*) as c, state, city from zips\n"
> > > +                    + "group by state, city\n"
> > > +                    + "order by c desc, city\n"
> > > +                    + "limit 2")
> > > +            .returns("C=1; STATE=SD; CITY=ABERDEEN\n"
> > > +                      + "C=1; STATE=SC; CITY=AIKEN\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {CITY: '$city', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> > '$STATE'}, C: {$sum: 1}}}",
> > > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> > STATE: '$_id.STATE', C: '$C'}}",
> > > +                            "{$sort: {C: -1, CITY: 1}}",
> > > +                            "{$limit: 2}",
> > > +                            "{$project: {C: 1, STATE: 1, CITY: 1}}"));
> > > +  }
> > > +
> > > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > > +  @Test public void testDistinctCount() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, count(distinct city) as cdc from
> > zips\n"
> > > +                    + "where state in ('CA', 'TX') group by state
> order
> > by state")
> > > +            .returns("STATE=CA; CDC=1072\n"
> > > +                    + "STATE=TX; CDC=1233\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{\n"
> > > +                                    + "  \"$match\": {\n"
> > > +                                    + "    \"$or\": [\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"state\": \"CA\"\n"
> > > +                                    + "      },\n"
> > > +                                    + "      {\n"
> > > +                                    + "        \"state\": \"TX\"\n"
> > > +                                    + "      }\n"
> > > +                                    + "    ]\n"
> > > +                                    + "  }\n"
> > > +                                    + "}",
> > > +                            "{$project: {CITY: '$city', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> > '$STATE'}}}",
> > > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> > STATE: '$_id.STATE'}}",
> > > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> > {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > > +                            "{$project: {STATE: '$_id', CDC:
> '$CDC'}}",
> > > +                            "{$sort: {STATE: 1}}"));
> > > +  }
> > > +
> > > +  @Test public void testDistinctCountOrderBy() {
> > > +    // java.lang.ClassCastException: com.mongodb.BasicDBObject cannot
> > be cast to java.lang.Number
> > > +    // https://github.com/fakemongo/fongo/issues/152
> > > +    MongoAssertions.assumeRealMongoInstance();
> > > +    assertModel(MODEL)
> > > +            .query("select state, count(distinct city) as cdc\n"
> > > +                    + "from zips\n"
> > > +                    + "group by state\n"
> > > +                    + "order by cdc desc limit 5")
> > > +            .returns("STATE=VA; CDC=3\n"
> > > +                    + "STATE=NY; CDC=3\n"
> > > +                    + "STATE=SC; CDC=3\n"
> > > +                    + "STATE=RI; CDC=3\n"
> > > +                    + "STATE=WV; CDC=3\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {CITY: '$city', STATE:
> > '$state'}}",
> > > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> > '$STATE'}}}",
> > > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> > STATE: '$_id.STATE'}}",
> > > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> > {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > > +                            "{$project: {STATE: '$_id', CDC:
> '$CDC'}}",
> > > +                            "{$sort: {CDC: -1}}",
> > > +                            "{$limit: 5}"));
> > > +  }
> > > +
> > > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > > +  @Test public void testProject() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, city, 0 as zero from zips order by
> > state, city")
> > > +            .limit(2)
> > > +            .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
> > > +                    + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
> > > +            .queryContains(
> > > +                    mongoChecker(
> > > +                            "{$project: {CITY: '$city', STATE:
> > '$state'}}",
> > > +                            "{$sort: {STATE: 1, CITY: 1}}",
> > > +                            "{$project: {STATE: 1, CITY: 1, ZERO:
> > {$literal: 0}}}"));
> > > +  }
> > > +
> > > +  @Test public void testFilter() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, city from zips where state = 'CA'")
> > > +            .limit(2)
> > > +            .returns("STATE=CA; CITY=LOS ANGELES\n"
> > > +                    + "STATE=CA; CITY=BELL GARDENS\n")
> > > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > > +                    + "  MongoProject(STATE=[CAST(ITEM($0,
> > 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20)
> > CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> > 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> > \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > > +                    + "      MongoTableScan(table=[[mongo_raw,
> > zips]])");
> > > +  }
> > > +
> > > +  /** MongoDB's predicates are handed (they can only accept literals
> on
> > the
> > > +   * right-hand size) so it's worth testing that we handle them right
> > both
> > > +   * ways around. */
> > > +  @Test public void testFilterReversed() {
> > > +    assertModel(MODEL)
> > > +            .query("select state, city from zips where 'WI' < state")
> > > +            .limit(2)
> > > +            .returns("STATE=WV; CITY=BECKLEY\nSTATE=WV; CITY=ELM
> > GROVE\n");
> > > +
> > > +    assertModel(MODEL)
> > > +            .query("select state, city from zips where state > 'WI'")
> > > +            .limit(2)
> > > +            .returns("STATE=WV; CITY=BECKLEY\n"
> > > +                    + "STATE=WV; CITY=ELM GROVE\n");
> > > +  }
> > > +
> > > +  /** MongoDB's predicates are handed (they can only accept literals
> on
> > the
> > > +   * right-hand size) so it's worth testing that we handle them right
> > both
> > > +   * ways around.
> > >    *
> > > -   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2331
> > ">[CALCITE-2331]</a>
> > > -   */
> > > -  @Ignore("broken; [CALCITE-2331] is logged to fix it")
> > > -  @Test
> > > -  public void validateCALCITE2331() {
> > > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> > "NY"));
> > > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> > "DC"));
> > > -
> > > -    CalciteAssert.that()
> > > -            .with(newConnectionFactory())
> > > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> > \"city\" from \"mongo\".\"zips\" "
> > > -                    + " where _MAP['STATE'] in ('DC', 'NY') and
> > _MAP['CITY'] = 'New York'")
> > > -            .returns("city=New York\n");
> > > +   * <p>Test case for
> > > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-740
> > ">[CALCITE-740]
> > > +   * Redundant WHERE clause causes wrong result in MongoDB
> adapter</a>.
> > */
> > > +  @Test public void testFilterPair() {
> > > +    final int gt9k = 148;
> > > +    final int lt9k = 1;
> > > +    final int gt8k = 148;
> > > +    final int lt8k = 1;
> > > +    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
> > > +    checkPredicate(gt9k, "where pop > 9000");
> > > +    checkPredicate(lt9k, "where pop < 9000");
> > > +    checkPredicate(gt8k, "where pop > 8000");
> > > +    checkPredicate(lt8k, "where pop < 8000");
> > > +    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
> > > +    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
> > > +    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
> > > +    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
> > > +  }
> > > +
> > > +  private void checkPredicate(int expected, String q) {
> > > +    assertModel(MODEL)
> > > +            .query("select count(*) as c from zips\n"
> > > +                    + q)
> > > +            .returns("C=" + expected + "\n");
> > > +    assertModel(MODEL)
> > > +            .query("select * from zips\n"
> > > +                    + q)
> > > +            .returnsCount(expected);
> > > +  }
> > > +
> > > +  /** Test case for
> > > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-286
> > ">[CALCITE-286]
> > > +   * Error casting MongoDB date</a>. */
> > > +  @Test public void testDate() {
> > > +    // Assumes that you have created the following collection before
> > running
> > > +    // this test:
> > > +    //
> > > +    // $ mongo
> > > +    // > use test
> > > +    // switched to db test
> > > +    // > db.createCollection("datatypes")
> > > +    // { "ok" : 1 }
> > > +    // > db.datatypes.insert( {
> > > +    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
> > > +    //     "_class" : "com.ericblue.Test",
> > > +    //     "date" : ISODate("2012-09-05T07:00:00Z"),
> > > +    //     "value" : 1231,
> > > +    //     "ownerId" : "531e7789e4b0853ddb861313"
> > > +    //   } )
> > > +    assertModel("{\n"
> > > +            + "  version: '1.0',\n"
> > > +            + "  defaultSchema: 'test',\n"
> > > +            + "   schemas: [\n"
> > > +            + "     {\n"
> > > +            + "       type: 'custom',\n"
> > > +            + "       name: 'test',\n"
> > > +            + "       factory:
> > 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> > > +            + "       operand: {\n"
> > > +            + "         host: 'localhost',\n"
> > > +            + "         database: 'test'\n"
> > > +            + "       }\n"
> > > +            + "     }\n"
> > > +            + "   ]\n"
> > > +            + "}")
> > > +            .query("select cast(_MAP['date'] as DATE) from
> > \"datatypes\"")
> > > +            .returnsUnordered("EXPR$0=2012-09-05");
> > > +  }
> > > +
> > > +  /** Test case for
> > > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-665
> > ">[CALCITE-665]
> > > +   * ClassCastException in MongoDB adapter</a>. */
> > > +  @Test public void testCountViaInt() {
> > > +    assertModel(MODEL)
> > > +        .query("select count(*) from zips")
> > > +        .returns(
> > > +            new Function<ResultSet, Void>() {
> > > +              public Void apply(ResultSet input) {
> > > +                try {
> > > +                  Assert.assertThat(input.next(),
> > CoreMatchers.is(true));
> > > +                  Assert.assertThat(input.getInt(1),
> > CoreMatchers.is(ZIPS_SIZE));
> > > +                  return null;
> > > +                } catch (SQLException e) {
> > > +                  throw new RuntimeException(e);
> > > +                }
> > > +              }
> > > +            });
> > > +  }
> > > +
> > > +  /** Returns a function that checks that a particular MongoDB
> pipeline
> > is
> > > +   * generated to implement a query. */
> > > +  private static Function<List, Void> mongoChecker(final String...
> > strings) {
> > > +    return new Function<List, Void>() {
> > > +      public Void apply(List actual) {
> > > +        Object[] actualArray =
> > > +                actual == null || actual.isEmpty()
> > > +                        ? null
> > > +                        : ((List) actual.get(0)).toArray();
> > > +        CalciteAssert.assertArrayEqual("expected MongoDB query not
> > found",
> > > +                strings, actualArray);
> > > +        return null;
> > > +      }
> > > +    };
> > >   }
> > > }
> > >
> > >
> > >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > > ----------------------------------------------------------------------
> > > diff --git
> >
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> >
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > > new file mode 100644
> > > index 0000000..90bd759
> > > --- /dev/null
> > > +++
> >
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > > @@ -0,0 +1,83 @@
> > > +/*
> > > + * 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.calcite.adapter.mongodb;
> > > +
> > > +import org.apache.calcite.test.MongoAssertions;
> > > +
> > > +import com.github.fakemongo.Fongo;
> > > +
> > > +import com.google.common.base.Preconditions;
> > > +
> > > +import com.mongodb.MongoClient;
> > > +import com.mongodb.client.MongoDatabase;
> > > +
> > > +import org.junit.rules.ExternalResource;
> > > +
> > > +/**
> > > + * Instantiates new connection to fongo (or mongo) database depending
> > on current profile
> > > + * (unit or integration tests).
> > > + *
> > > + * By default, this rule is executed as part of a unit test and
> > in-memory database
> > > + * <a href="https://github.com/fakemongo/fongo">fongo</a> is used.
> > > + *
> > > + * <p>However, if maven profile is set to {@code IT} (eg. via command
> > line
> > > + * {@code $ mvn -Pit install}) this rule will connect to existing
> > (external)
> > > + * mongo instance ({@code localhost})</p>
> > > + *
> > > + */
> > > +class MongoDatabaseRule extends ExternalResource {
> > > +
> > > +  private static final String DB_NAME = "test";
> > > +
> > > +  private final MongoDatabase database;
> > > +  private final MongoClient client;
> > > +
> > > +  private MongoDatabaseRule(MongoClient client) {
> > > +    this.client = Preconditions.checkNotNull(client, "client");
> > > +    this.database = client.getDatabase(DB_NAME);
> > > +  }
> > > +
> > > +  /**
> > > +   * Create an instance based on current maven profile (as defined by
> > {@code -Pit}).
> > > +   */
> > > +  static MongoDatabaseRule create() {
> > > +    final MongoClient client;
> > > +    if (MongoAssertions.useMongo()) {
> > > +      // use to real client (connects to mongo)
> > > +      client = new MongoClient();
> > > +    } else if (MongoAssertions.useFongo()) {
> > > +      // in-memory DB (fake Mongo)
> > > +

Re: [2/2] calcite git commit: [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

Posted by Michael Mior <mm...@apache.org>.
Done. Thanks for consistently running those tests.
--
Michael Mior
mmior@apache.org


Le jeu. 14 juin 2018 à 10:42, Julian Hyde <jh...@apache.org> a écrit :

> This change introduces a javadoc error:
>
> [ERROR]
> /home/jhyde/regress/calcite/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java:35:
> error: unexpected end tag: </p>
> [ERROR]  * </p>
>
> Please fix ASAP.
>
>
> > On Jun 13, 2018, at 6:00 PM, mmior@apache.org wrote:
> >
> > [CALCITE-2345] Running Unit tests with Fongo and integration tests with
> real mongo instance (Andrei Sereda)
> >
> > Better test coverage for unit tests using Fongo in-memory implementation
> of Mongo API.
> > New code will decide (at runtime) what connection to make: fongo vs
> mongo. Identical tests will be run against
> > both databases (depending on maven profile surefire vs failsafe)
> >
> > Close apache/calcite#723
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
> > Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/dcf396a5
> > Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/dcf396a5
> > Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/dcf396a5
> >
> > Branch: refs/heads/master
> > Commit: dcf396a5ca92ee0714c28d9a3c310f53f46220ab
> > Parents: f84a3eb
> > Author: Andrei Sereda <an...@nospam.com>
> > Authored: Tue Jun 5 16:23:56 2018 -0400
> > Committer: Michael Mior <mm...@uwaterloo.ca>
> > Committed: Wed Jun 13 18:59:15 2018 -0400
> >
> > ----------------------------------------------------------------------
> > mongodb/pom.xml                                 |  10 +-
> > .../adapter/mongodb/MongoAdapterTest.java       | 794 +++++++++++++++---
> > .../adapter/mongodb/MongoDatabaseRule.java      |  83 ++
> > .../org/apache/calcite/test/MongoAdapterIT.java | 830 +------------------
> > .../apache/calcite/test/MongoAssertions.java    | 101 +++
> > .../test/resources/mongo-foodmart-model.json    | 221 -----
> > mongodb/src/test/resources/mongo-model.json     |  70 ++
> > .../src/test/resources/mongo-zips-model.json    |  41 -
> > mongodb/src/test/resources/zips-mini.json       | 149 ++++
> > pom.xml                                         |   9 +
> > 10 files changed, 1132 insertions(+), 1176 deletions(-)
> > ----------------------------------------------------------------------
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/pom.xml
> > ----------------------------------------------------------------------
> > diff --git a/mongodb/pom.xml b/mongodb/pom.xml
> > index 25c0642..63cc147 100644
> > --- a/mongodb/pom.xml
> > +++ b/mongodb/pom.xml
> > @@ -71,6 +71,11 @@ limitations under the License.
> >       <scope>test</scope>
> >     </dependency>
> >     <dependency>
> > +      <groupId>net.hydromatic</groupId>
> > +      <artifactId>foodmart-data-json</artifactId>
> > +      <scope>test</scope>
> > +    </dependency>
> > +    <dependency>
> >       <groupId>org.mongodb</groupId>
> >       <artifactId>mongo-java-driver</artifactId>
> >     </dependency>
> > @@ -78,11 +83,6 @@ limitations under the License.
> >       <groupId>org.slf4j</groupId>
> >       <artifactId>slf4j-api</artifactId>
> >     </dependency>
> > -    <dependency>
> > -      <groupId>org.slf4j</groupId>
> > -      <artifactId>slf4j-log4j12</artifactId>
> > -      <scope>test</scope>
> > -    </dependency>
> >   </dependencies>
> >
> >   <build>
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > index b6ca4f3..a4061e4 100644
> > ---
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > +++
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> > @@ -16,143 +16,749 @@
> >  */
> > package org.apache.calcite.adapter.mongodb;
> >
> > -import org.apache.calcite.jdbc.CalciteConnection;
> > +import org.apache.calcite.schema.Schema;
> > +import org.apache.calcite.schema.SchemaFactory;
> > import org.apache.calcite.schema.SchemaPlus;
> > import org.apache.calcite.test.CalciteAssert;
> > +import org.apache.calcite.test.MongoAssertions;
> >
> > -import com.github.fakemongo.junit.FongoRule;
> > +
> > +import org.apache.calcite.util.Bug;
> > +import org.apache.calcite.util.Util;
> > +
> > +import com.google.common.base.Function;
> > +import com.google.common.base.Preconditions;
> > +import com.google.common.io.LineProcessor;
> > +import com.google.common.io.Resources;
> > import com.mongodb.client.MongoCollection;
> > import com.mongodb.client.MongoDatabase;
> >
> > +import net.hydromatic.foodmart.data.json.FoodmartJson;
> > +
> > +import org.bson.BsonDateTime;
> > import org.bson.BsonDocument;
> > +import org.bson.BsonInt32;
> > +import org.bson.BsonString;
> > import org.bson.Document;
> >
> > -import org.junit.Before;
> > +import org.hamcrest.CoreMatchers;
> > +
> > +import org.junit.Assert;
> > +import org.junit.BeforeClass;
> > +import org.junit.ClassRule;
> > import org.junit.Ignore;
> > -import org.junit.Rule;
> > import org.junit.Test;
> >
> > -import java.sql.Connection;
> > -import java.sql.DriverManager;
> > +import java.io.IOException;
> > +import java.io.UncheckedIOException;
> > +import java.net.URL;
> > +import java.nio.charset.StandardCharsets;
> > +import java.sql.ResultSet;
> > import java.sql.SQLException;
> > +import java.text.SimpleDateFormat;
> > +import java.util.Date;
> > +import java.util.List;
> > +import java.util.Locale;
> > +import java.util.Map;
> >
> > /**
> > - * Tests current adapter using in-memory (fake) implementation of Mongo
> API:
> > - * <a href="https://github.com/fakemongo/fongo">Fongo</a>.
> > - *
> > + * Testing mongo adapter functionality. By default runs with
> > + * <a href="https://github.com/fakemongo/fongo">Fongo</a> unless
> {@code IT} maven profile is enabled
> > + * (via {@code $ mvn -Pit install}).
> >  */
> > -public class MongoAdapterTest {
> > +public class MongoAdapterTest implements SchemaFactory {
> > +
> > +  /** Connection factory based on the "mongo-zips" model. */
> > +  protected static final URL MODEL =
> MongoAdapterTest.class.getResource("/mongo-model.json");
> > +
> > +  /** Number of records in local file */
> > +  protected static final int ZIPS_SIZE = 149;
> > +
> > +  @ClassRule
> > +  public static final MongoDatabaseRule RULE =
> MongoDatabaseRule.create();
> > +
> > +  private static MongoSchema schema;
> > +
> > +  @BeforeClass
> > +  public static void setUp() throws Exception {
> > +    MongoDatabase database = RULE.database();
> > +
> > +    populate(database.getCollection("zips"),
> MongoAdapterTest.class.getResource("/zips-mini.json"));
> > +    populate(database.getCollection("store"),
> FoodmartJson.class.getResource("/store.json"));
> > +    populate(database.getCollection("warehouse"),
> > +            FoodmartJson.class.getResource("/warehouse.json"));
> >
> > -  @Rule
> > -  public final FongoRule rule = new FongoRule();
> > +    // Manually insert data for data-time test.
> > +    MongoCollection<BsonDocument> datatypes =
> database.getCollection("datatypes")
> > +            .withDocumentClass(BsonDocument.class);
> > +    if (datatypes.count() > 0) {
> > +      datatypes.deleteMany(new BsonDocument());
> > +    }
> > +    BsonDocument doc = new BsonDocument();
> > +    Date date = new SimpleDateFormat("yyyy-MM-dd",
> Locale.getDefault()).parse("2012-09-05");
> > +    doc.put("date", new BsonDateTime(date.getTime()));
> > +    doc.put("value", new BsonInt32(1231));
> > +    doc.put("ownerId", new BsonString("531e7789e4b0853ddb861313"));
> > +    datatypes.insertOne(doc);
> >
> > -  private MongoDatabase mongoDb;
> > -  private MongoCollection<Document> zips;
> > +    schema = new MongoSchema(database);
> > +  }
> > +
> > +  private static void populate(MongoCollection<Document> collection,
> URL resource)
> > +          throws IOException {
> > +    Preconditions.checkNotNull(collection, "collection");
> > +
> > +    if (collection.count() > 0) {
> > +      // delete any existing documents (run from a clean set)
> > +      collection.deleteMany(new BsonDocument());
> > +    }
> >
> > -  @Before
> > -  public void setUp() throws Exception {
> > -    mongoDb = rule.getDatabase(getClass().getSimpleName());
> > -    zips = mongoDb.getCollection("zips");
> > +    MongoCollection<BsonDocument> bsonCollection =
> collection.withDocumentClass(BsonDocument.class);
> > +    Resources.readLines(resource, StandardCharsets.UTF_8, new
> LineProcessor<Void>() {
> > +      @Override public boolean processLine(String line) throws
> IOException {
> > +        bsonCollection.insertOne(BsonDocument.parse(line));
> > +        return true;
> > +      }
> > +
> > +      @Override public Void getResult() {
> > +        return null;
> > +      }
> > +    });
> >   }
> >
> >   /**
> > -   * Handcrafted connection where we manually added {@link MongoSchema}
> > +   *  Returns always the same schema to avoid initialization costs.
> >    */
> > -  private CalciteAssert.ConnectionFactory newConnectionFactory() {
> > -    return new CalciteAssert.ConnectionFactory() {
> > -      @Override public Connection createConnection() throws
> SQLException {
> > -        Connection connection =
> DriverManager.getConnection("jdbc:calcite:");
> > -        final SchemaPlus root =
> connection.unwrap(CalciteConnection.class).getRootSchema();
> > -        root.add("mongo", new MongoSchema(mongoDb));
> > -        return connection;
> > -      }
> > -    };
> > +  @Override public Schema create(SchemaPlus parentSchema, String name,
> > +                                 Map<String, Object> operand) {
> > +    return schema;
> > +  }
> > +
> > +  private CalciteAssert.AssertThat assertModel(String model) {
> > +    // ensure that Schema from this instance is being used
> > +    model = model.replace(MongoSchemaFactory.class.getName(),
> MongoAdapterTest.class.getName());
> > +
> > +    return CalciteAssert.that()
> > +            .withModel(model);
> > +  }
> > +
> > +  private CalciteAssert.AssertThat assertModel(URL url) {
> > +    Preconditions.checkNotNull(url, "url");
> > +    try {
> > +      return assertModel(Resources.toString(url,
> StandardCharsets.UTF_8));
> > +    } catch (IOException e) {
> > +      throw new UncheckedIOException(e);
> > +    }
> >   }
> >
> >   @Test
> > -  public void single() {
> > -    zips.insertOne(new Document());
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\"")
> > +  public void testSort() {
> > +    assertModel(MODEL)
> > +            .query("select * from zips order by state")
> > +            .returnsCount(ZIPS_SIZE)
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
> > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> > +
> > +  @Test public void testSortLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "order by state, id offset 2 rows fetch next 3
> rows only")
> > +            .returns("STATE=AK; ID=99801\n"
> > +                    + "STATE=AL; ID=35215\n"
> > +                    + "STATE=AL; ID=35401\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state', ID: '$_id'}}",
> > +                            "{$sort: {STATE: 1, ID: 1}}",
> > +                            "{$skip: 2}",
> > +                            "{$limit: 3}"));
> > +  }
> > +
> > +  @Test public void testOffsetLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "offset 2 fetch next 3 rows only")
> >             .runs()
> > -            .returnsCount(1);
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$skip: 2}",
> > +                            "{$limit: 3}",
> > +                            "{$project: {STATE: '$state', ID:
> '$_id'}}"));
> >   }
> >
> > -  @Test
> > -  public void empty() {
> > -    // for some reason fongo doesn't list collection if it was unused
> > -    zips.insertOne(new Document());
> > -    zips.deleteMany(new BsonDocument());
> > -
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\"")
> > +  @Test public void testLimit() {
> > +    assertModel(MODEL)
> > +            .query("select state, id from zips\n"
> > +                    + "fetch next 3 rows only")
> >             .runs()
> > -            .returnsCount(0);
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$limit: 3}",
> > +                            "{$project: {STATE: '$state', ID:
> '$_id'}}"));
> >   }
> >
> > -  @Test
> > -  public void filter() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > +  @Ignore
> > +  @Test public void testFilterSort() {
> > +    // LONGITUDE and LATITUDE are null because of CALCITE-194.
> > +    Util.discard(Bug.CALCITE_194_FIXED);
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + "where city = 'SPRINGFIELD' and id >= '70000'\n"
> > +                    + "order by state, id")
> > +            .returns(""
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=752; STATE=AR; ID=72157\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=1992; STATE=CO; ID=81073\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=5597; STATE=LA; ID=70462\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=32384; STATE=OR; ID=97477\n"
> > +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null;
> POP=27521; STATE=OR; ID=97478\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  $match: {\n"
> > +                                    + "    city: \"SPRINGFIELD\",\n"
> > +                                    + "    _id: {\n"
> > +                                    + "      $gte: \"70000\"\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', LONGITUDE:
> '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
> > +                            "{$sort: {STATE: 1, ID: 1}}"))
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC],
> dir1=[ASC])\n"
> > +                    + "    MongoProject(CITY=[CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'),
> 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT],
> POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "      MongoFilter(condition=[AND(=(CAST(ITEM($0,
> 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0,
> '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", '70000'))])\n"
> > +                    + "        MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] = 'NY'")
> > -            .returns("city=New York\n");
> > +  @Test public void testFilterSortDesc() {
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + "where pop BETWEEN 45000 AND 46000\n"
> > +                    + "order by state desc, pop")
> > +            .limit(4)
> > +            .returns("CITY=BECKLEY; LONGITUDE=null; LATITUDE=null;
> POP=45196; STATE=WV; ID=25801\n"
> > +                    + "CITY=ROCKERVILLE; LONGITUDE=null; LATITUDE=null;
> POP=45328; STATE=SD; ID=57701\n"
> > +                    + "CITY=PAWTUCKET; LONGITUDE=null; LATITUDE=null;
> POP=45442; STATE=RI; ID=02860\n"
> > +                    + "CITY=LAWTON; LONGITUDE=null; LATITUDE=null;
> POP=45542; STATE=OK; ID=73505\n");
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                   + " where _MAP['STATE'] = 'DC'")
> > -            .returns("city=Washington\n");
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testUnionPlan() {
> > +    assertModel(MODEL)
> > +            .query("select * from \"sales_fact_1997\"\n"
> > +                    + "union all\n"
> > +                    + "select * from \"sales_fact_1998\"")
> > +            .explainContains("PLAN=EnumerableUnion(all=[true])\n"
> > +                    + "  MongoToEnumerableConverter\n"
> > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> 'product_id')):DOUBLE])\n"
> > +                    + "      MongoTableScan(table=[[_foodmart,
> sales_fact_1997]])\n"
> > +                    + "  MongoToEnumerableConverter\n"
> > +                    + "    MongoProject(product_id=[CAST(ITEM($0,
> 'product_id')):DOUBLE])\n"
> > +                    + "      MongoTableScan(table=[[_foodmart,
> sales_fact_1998]])")
> > +            .limit(2)
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "product_id=337", "product_id=1512"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] in ('DC', 'NY')")
> > -            .returns("city=New York\ncity=Washington\n");
> > +  @Ignore(
> > +          "java.lang.ClassCastException: java.lang.Integer cannot be
> cast to java.lang.Double")
> > +  @Test public void testFilterUnionPlan() {
> > +    assertModel(MODEL)
> > +            .query("select * from (\n"
> > +                    + "  select * from \"sales_fact_1997\"\n"
> > +                    + "  union all\n"
> > +                    + "  select * from \"sales_fact_1998\")\n"
> > +                    + "where \"product_id\" = 1")
> > +            .runs();
> >   }
> >
> > -  @Test
> > -  public void limit() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > +  /** Tests that we don't generate multiple constraints on the same
> column.
> > +   * MongoDB doesn't like it. If there is an '=', it supersedes all
> other
> > +   * operators. */
> > +  @Test public void testFilterRedundant() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select * from zips where state > 'CA' and state <
> 'AZ' and state = 'OK'")
> > +            .runs()
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"state\": \"OK\"\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', LONGITUDE:
> '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID:
> '$_id'}}"));
> > +  }
> > +
> > +  @Test public void testSelectWhere() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select * from \"warehouse\" where
> \"warehouse_state_province\" = 'CA'")
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoProject(warehouse_id=[CAST(ITEM($0,
> 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0,
> 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\"
> COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> warehouse]])")
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "warehouse_id=6;
> warehouse_state_province=CA",
> > +                            "warehouse_id=7;
> warehouse_state_province=CA",
> > +                            "warehouse_id=14;
> warehouse_state_province=CA",
> > +                            "warehouse_id=24;
> warehouse_state_province=CA"))
> > +            .queryContains(
> > +                    // Per
> https://issues.apache.org/jira/browse/CALCITE-164,
> > +                    // $match must occur before $project for good
> performance.
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "
> \"warehouse_state_province\": \"CA\"\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {warehouse_id: 1,
> warehouse_state_province: 1}}"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\" limit 1")
> > -            .returnsCount(1);
> > +  @Test public void testInPlan() {
> > +    assertModel(MODEL)
> > +            .query("select \"store_id\", \"store_name\" from
> \"store\"\n"
> > +                    + "where \"store_name\" in ('Store 1', 'Store 10',
> 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
> > +            .returns(
> > +                    MongoAssertions.checkResultUnordered(
> > +                            "store_id=1; store_name=Store 1",
> > +                            "store_id=3; store_name=Store 3",
> > +                            "store_id=7; store_name=Store 7",
> > +                            "store_id=10; store_name=Store 10",
> > +                            "store_id=11; store_name=Store 11",
> > +                            "store_id=15; store_name=Store 15",
> > +                            "store_id=16; store_name=Store 16",
> > +                            "store_id=24; store_name=Store 24"))
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"$or\": [\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 1\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 10\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 11\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 15\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 16\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 24\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 3\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"store_name\": \"Store
> 7\"\n"
> > +                                    + "      }\n"
> > +                                    + "    ]\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {store_id: 1, store_name:
> 1}}"));
> > +  }
> >
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select * from \"mongo\".\"zips\" limit 2")
> > -            .returnsCount(2);
> > +  /** Simple query based on the "mongo-zips" model. */
> > +  @Test public void testZips() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips")
> > +            .returnsCount(ZIPS_SIZE);
> > +  }
> >
> > +  @Test public void testCountGroupByEmpty() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) from zips")
> > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> ZIPS_SIZE))
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
> > +                    + "    MongoTableScan(table=[[mongo_raw, zips]])")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$group: {_id: {}, 'EXPR$0': {$sum:
> 1}}}"));
> >   }
> >
> > -  /**
> > -   * Following queries are not supported in Mongo adapter :
> > -   * <pre>
> > -   * {@code A and (B or C)}
> > -   * {@code (A or B) and C}
> > -   * </pre>
> > +  @Test public void testCountGroupByEmptyMultiplyBy2() {
> > +    // This operation is not supported by fongo:
> https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +
> > +    assertModel(MODEL)
> > +            .query("select count(*)*2 from zips")
> > +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n",
> ZIPS_SIZE * 2))
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$group: {_id: {}, _0: {$sum: 1}}}",
> > +                            "{$project: {'EXPR$0': {$multiply: ['$_0',
> {$literal: 2}]}}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumnNotProjected() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) from zips group by state order by
> 1")
> > +            .limit(2)
> > +            .returns("EXPR$0=2\n"
> > +                    + "EXPR$0=2\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', 'EXPR$0': {$sum:
> 1}}}",
> > +                            "{$project: {STATE: '$_id', 'EXPR$0':
> '$EXPR$0'}}",
> > +                            "{$project: {'EXPR$0': 1}}",
> > +                            "{$sort: {EXPR$0: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumn() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, count(*) as c from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; C=3\nSTATE=AL; C=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByOneColumnReversed() {
> > +    // Note extra $project compared to testGroupByOneColumn.
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select count(*) as c, state from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("C=3; STATE=AK\nC=3; STATE=AL\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{$project: {C: 1, STATE: 1}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByAvg() {
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, avg(pop) as a from zips group by
> state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; A=26856\nSTATE=AL; A=43383\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', A: {$avg:
> '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', A: '$A'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByAvgSumCount() {
> > +    // This operation not supported by fongo:
> https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +    assertModel(MODEL)
> > +            .query(
> > +                    "select state, avg(pop) as a, sum(pop) as s,
> count(pop) as c from zips group by state order by state")
> > +            .limit(2)
> > +            .returns("STATE=AK; A=26856; S=80568; C=3\n"
> > +                    + "STATE=AL; A=43383; S=130151; C=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', _1: {$sum:
> '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', _1: '$_1', _2:
> '$_2'}}",
> > +                            "{$sort: {STATE: 1}}",
> > +                            "{$project: {STATE: 1, A: {$divide:
> [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S:
> {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
> > +  }
> > +
> > +  @Test public void testGroupByHaving() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(*) as c from zips\n"
> > +                    + "group by state having count(*) > 2 order by
> state")
> > +            .returnsCount(47)
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C'}}",
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"C\": {\n"
> > +                                    + "      \"$gt\": 2\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
> > +  @Test public void testGroupByHaving2() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(*) as c from zips\n"
> > +                    + "group by state having sum(pop) > 12000000")
> > +            .returns("STATE=NY; C=1596\n"
> > +                    + "STATE=TX; C=1676\n"
> > +                    + "STATE=FL; C=826\n"
> > +                    + "STATE=CA; C=1523\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {STATE: '$state', POP:
> '$pop'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1}, _2:
> {$sum: '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C', _2:
> '$_2'}}",
> > +                            "{\n"
> > +                                    + "  $match: {\n"
> > +                                    + "    _2: {\n"
> > +                                    + "      $gt: 12000000\n"
> > +                                    + "    }\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {STATE: 1, C: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupByMinMaxSum() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c, state,\n"
> > +                    + " min(pop) as min_pop, max(pop) as max_pop,
> sum(pop) as sum_pop\n"
> > +                    + "from zips group by state order by state")
> > +            .limit(2)
> > +            .returns("C=3; STATE=AK; MIN_POP=23238; MAX_POP=32383;
> SUM_POP=80568\n"
> > +                    + "C=3; STATE=AL; MIN_POP=42124; MAX_POP=44165;
> SUM_POP=130151\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {POP: '$pop', STATE:
> '$state'}}",
> > +                            "{$group: {_id: '$STATE', C: {$sum: 1},
> MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum:
> '$POP'}}}",
> > +                            "{$project: {STATE: '$_id', C: '$C',
> MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
> > +                            "{$project: {C: 1, STATE: 1, MIN_POP: 1,
> MAX_POP: 1, SUM_POP: 1}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testGroupComposite() {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c, state, city from zips\n"
> > +                    + "group by state, city\n"
> > +                    + "order by c desc, city\n"
> > +                    + "limit 2")
> > +            .returns("C=1; STATE=SD; CITY=ABERDEEN\n"
> > +                      + "C=1; STATE=SC; CITY=AIKEN\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}, C: {$sum: 1}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE', C: '$C'}}",
> > +                            "{$sort: {C: -1, CITY: 1}}",
> > +                            "{$limit: 2}",
> > +                            "{$project: {C: 1, STATE: 1, CITY: 1}}"));
> > +  }
> > +
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testDistinctCount() {
> > +    assertModel(MODEL)
> > +            .query("select state, count(distinct city) as cdc from
> zips\n"
> > +                    + "where state in ('CA', 'TX') group by state order
> by state")
> > +            .returns("STATE=CA; CDC=1072\n"
> > +                    + "STATE=TX; CDC=1233\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{\n"
> > +                                    + "  \"$match\": {\n"
> > +                                    + "    \"$or\": [\n"
> > +                                    + "      {\n"
> > +                                    + "        \"state\": \"CA\"\n"
> > +                                    + "      },\n"
> > +                                    + "      {\n"
> > +                                    + "        \"state\": \"TX\"\n"
> > +                                    + "      }\n"
> > +                                    + "    ]\n"
> > +                                    + "  }\n"
> > +                                    + "}",
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE'}}",
> > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> > +                            "{$sort: {STATE: 1}}"));
> > +  }
> > +
> > +  @Test public void testDistinctCountOrderBy() {
> > +    // java.lang.ClassCastException: com.mongodb.BasicDBObject cannot
> be cast to java.lang.Number
> > +    // https://github.com/fakemongo/fongo/issues/152
> > +    MongoAssertions.assumeRealMongoInstance();
> > +    assertModel(MODEL)
> > +            .query("select state, count(distinct city) as cdc\n"
> > +                    + "from zips\n"
> > +                    + "group by state\n"
> > +                    + "order by cdc desc limit 5")
> > +            .returns("STATE=VA; CDC=3\n"
> > +                    + "STATE=NY; CDC=3\n"
> > +                    + "STATE=SC; CDC=3\n"
> > +                    + "STATE=RI; CDC=3\n"
> > +                    + "STATE=WV; CDC=3\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$group: {_id: {CITY: '$CITY', STATE:
> '$STATE'}}}",
> > +                            "{$project: {_id: 0, CITY: '$_id.CITY',
> STATE: '$_id.STATE'}}",
> > +                            "{$group: {_id: '$STATE', CDC: {$sum:
> {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> > +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> > +                            "{$sort: {CDC: -1}}",
> > +                            "{$limit: 5}"));
> > +  }
> > +
> > +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> > +  @Test public void testProject() {
> > +    assertModel(MODEL)
> > +            .query("select state, city, 0 as zero from zips order by
> state, city")
> > +            .limit(2)
> > +            .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
> > +                    + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
> > +            .queryContains(
> > +                    mongoChecker(
> > +                            "{$project: {CITY: '$city', STATE:
> '$state'}}",
> > +                            "{$sort: {STATE: 1, CITY: 1}}",
> > +                            "{$project: {STATE: 1, CITY: 1, ZERO:
> {$literal: 0}}}"));
> > +  }
> > +
> > +  @Test public void testFilter() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where state = 'CA'")
> > +            .limit(2)
> > +            .returns("STATE=CA; CITY=LOS ANGELES\n"
> > +                    + "STATE=CA; CITY=BELL GARDENS\n")
> > +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> > +                    + "  MongoProject(STATE=[CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20)
> CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> > +                    + "    MongoFilter(condition=[=(CAST(ITEM($0,
> 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE
> \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> > +                    + "      MongoTableScan(table=[[mongo_raw,
> zips]])");
> > +  }
> > +
> > +  /** MongoDB's predicates are handed (they can only accept literals on
> the
> > +   * right-hand size) so it's worth testing that we handle them right
> both
> > +   * ways around. */
> > +  @Test public void testFilterReversed() {
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where 'WI' < state")
> > +            .limit(2)
> > +            .returns("STATE=WV; CITY=BECKLEY\nSTATE=WV; CITY=ELM
> GROVE\n");
> > +
> > +    assertModel(MODEL)
> > +            .query("select state, city from zips where state > 'WI'")
> > +            .limit(2)
> > +            .returns("STATE=WV; CITY=BECKLEY\n"
> > +                    + "STATE=WV; CITY=ELM GROVE\n");
> > +  }
> > +
> > +  /** MongoDB's predicates are handed (they can only accept literals on
> the
> > +   * right-hand size) so it's worth testing that we handle them right
> both
> > +   * ways around.
> >    *
> > -   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2331
> ">[CALCITE-2331]</a>
> > -   */
> > -  @Ignore("broken; [CALCITE-2331] is logged to fix it")
> > -  @Test
> > -  public void validateCALCITE2331() {
> > -    zips.insertOne(new Document("CITY", "New York").append("STATE",
> "NY"));
> > -    zips.insertOne(new Document("CITY", "Washington").append("STATE",
> "DC"));
> > -
> > -    CalciteAssert.that()
> > -            .with(newConnectionFactory())
> > -            .query("select cast(_MAP['CITY'] as varchar(20)) as
> \"city\" from \"mongo\".\"zips\" "
> > -                    + " where _MAP['STATE'] in ('DC', 'NY') and
> _MAP['CITY'] = 'New York'")
> > -            .returns("city=New York\n");
> > +   * <p>Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-740
> ">[CALCITE-740]
> > +   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>.
> */
> > +  @Test public void testFilterPair() {
> > +    final int gt9k = 148;
> > +    final int lt9k = 1;
> > +    final int gt8k = 148;
> > +    final int lt8k = 1;
> > +    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
> > +    checkPredicate(gt9k, "where pop > 9000");
> > +    checkPredicate(lt9k, "where pop < 9000");
> > +    checkPredicate(gt8k, "where pop > 8000");
> > +    checkPredicate(lt8k, "where pop < 8000");
> > +    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
> > +    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
> > +    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
> > +    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
> > +  }
> > +
> > +  private void checkPredicate(int expected, String q) {
> > +    assertModel(MODEL)
> > +            .query("select count(*) as c from zips\n"
> > +                    + q)
> > +            .returns("C=" + expected + "\n");
> > +    assertModel(MODEL)
> > +            .query("select * from zips\n"
> > +                    + q)
> > +            .returnsCount(expected);
> > +  }
> > +
> > +  /** Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-286
> ">[CALCITE-286]
> > +   * Error casting MongoDB date</a>. */
> > +  @Test public void testDate() {
> > +    // Assumes that you have created the following collection before
> running
> > +    // this test:
> > +    //
> > +    // $ mongo
> > +    // > use test
> > +    // switched to db test
> > +    // > db.createCollection("datatypes")
> > +    // { "ok" : 1 }
> > +    // > db.datatypes.insert( {
> > +    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
> > +    //     "_class" : "com.ericblue.Test",
> > +    //     "date" : ISODate("2012-09-05T07:00:00Z"),
> > +    //     "value" : 1231,
> > +    //     "ownerId" : "531e7789e4b0853ddb861313"
> > +    //   } )
> > +    assertModel("{\n"
> > +            + "  version: '1.0',\n"
> > +            + "  defaultSchema: 'test',\n"
> > +            + "   schemas: [\n"
> > +            + "     {\n"
> > +            + "       type: 'custom',\n"
> > +            + "       name: 'test',\n"
> > +            + "       factory:
> 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> > +            + "       operand: {\n"
> > +            + "         host: 'localhost',\n"
> > +            + "         database: 'test'\n"
> > +            + "       }\n"
> > +            + "     }\n"
> > +            + "   ]\n"
> > +            + "}")
> > +            .query("select cast(_MAP['date'] as DATE) from
> \"datatypes\"")
> > +            .returnsUnordered("EXPR$0=2012-09-05");
> > +  }
> > +
> > +  /** Test case for
> > +   * <a href="https://issues.apache.org/jira/browse/CALCITE-665
> ">[CALCITE-665]
> > +   * ClassCastException in MongoDB adapter</a>. */
> > +  @Test public void testCountViaInt() {
> > +    assertModel(MODEL)
> > +        .query("select count(*) from zips")
> > +        .returns(
> > +            new Function<ResultSet, Void>() {
> > +              public Void apply(ResultSet input) {
> > +                try {
> > +                  Assert.assertThat(input.next(),
> CoreMatchers.is(true));
> > +                  Assert.assertThat(input.getInt(1),
> CoreMatchers.is(ZIPS_SIZE));
> > +                  return null;
> > +                } catch (SQLException e) {
> > +                  throw new RuntimeException(e);
> > +                }
> > +              }
> > +            });
> > +  }
> > +
> > +  /** Returns a function that checks that a particular MongoDB pipeline
> is
> > +   * generated to implement a query. */
> > +  private static Function<List, Void> mongoChecker(final String...
> strings) {
> > +    return new Function<List, Void>() {
> > +      public Void apply(List actual) {
> > +        Object[] actualArray =
> > +                actual == null || actual.isEmpty()
> > +                        ? null
> > +                        : ((List) actual.get(0)).toArray();
> > +        CalciteAssert.assertArrayEqual("expected MongoDB query not
> found",
> > +                strings, actualArray);
> > +        return null;
> > +      }
> > +    };
> >   }
> > }
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > new file mode 100644
> > index 0000000..90bd759
> > --- /dev/null
> > +++
> b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> > @@ -0,0 +1,83 @@
> > +/*
> > + * 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.calcite.adapter.mongodb;
> > +
> > +import org.apache.calcite.test.MongoAssertions;
> > +
> > +import com.github.fakemongo.Fongo;
> > +
> > +import com.google.common.base.Preconditions;
> > +
> > +import com.mongodb.MongoClient;
> > +import com.mongodb.client.MongoDatabase;
> > +
> > +import org.junit.rules.ExternalResource;
> > +
> > +/**
> > + * Instantiates new connection to fongo (or mongo) database depending
> on current profile
> > + * (unit or integration tests).
> > + *
> > + * By default, this rule is executed as part of a unit test and
> in-memory database
> > + * <a href="https://github.com/fakemongo/fongo">fongo</a> is used.
> > + *
> > + * <p>However, if maven profile is set to {@code IT} (eg. via command
> line
> > + * {@code $ mvn -Pit install}) this rule will connect to existing
> (external)
> > + * mongo instance ({@code localhost})</p>
> > + *
> > + */
> > +class MongoDatabaseRule extends ExternalResource {
> > +
> > +  private static final String DB_NAME = "test";
> > +
> > +  private final MongoDatabase database;
> > +  private final MongoClient client;
> > +
> > +  private MongoDatabaseRule(MongoClient client) {
> > +    this.client = Preconditions.checkNotNull(client, "client");
> > +    this.database = client.getDatabase(DB_NAME);
> > +  }
> > +
> > +  /**
> > +   * Create an instance based on current maven profile (as defined by
> {@code -Pit}).
> > +   */
> > +  static MongoDatabaseRule create() {
> > +    final MongoClient client;
> > +    if (MongoAssertions.useMongo()) {
> > +      // use to real client (connects to mongo)
> > +      client = new MongoClient();
> > +    } else if (MongoAssertions.useFongo()) {
> > +      // in-memory DB (fake Mongo)
> > +      client = new
> Fongo(MongoDatabaseRule.class.getSimpleName()).getMongo();
> > +    } else {
> > +      throw new UnsupportedOperationException("I can only connect to
> Mongo or Fongo instances");
> > +    }
> > +
> > +    return new MongoDatabaseRule(client);
> > +  }
> > +
> > +
> > +  MongoDatabase database() {
> > +    return database;
> > +  }
> > +
> > +  @Override protected void after() {
> > +    client.close();
> > +  }
> > +
> > +}
> > +
> > +// End MongoDatabaseRule.java
> >
> >
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > ----------------------------------------------------------------------
> > diff --git
> a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > index 00ed03a..bda1163 100644
> > --- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > +++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> > @@ -16,829 +16,29 @@
> >  */
> > package org.apache.calcite.test;
> >
> > -import org.apache.calcite.linq4j.Ord;
> > -import org.apache.calcite.util.Bug;
> > -import org.apache.calcite.util.Pair;
> > -import org.apache.calcite.util.Util;
> > +import org.apache.calcite.adapter.mongodb.MongoAdapterTest;
> >
> > -import com.google.common.base.Function;
> > -import com.google.common.collect.ImmutableMap;
> > -import com.google.common.collect.Lists;
> > -import com.google.common.collect.Ordering;
> > +import org.junit.BeforeClass;
> >
> > -import org.hamcrest.CoreMatchers;
> > -import org.junit.Ignore;
> > -import org.junit.Test;
> > -
> > -import java.sql.ResultSet;
> > -import java.sql.SQLException;
> > -import java.util.Arrays;
> > -import java.util.Collections;
> > -import java.util.List;
> > -
> > -import static org.hamcrest.CoreMatchers.equalTo;
> > -import static org.junit.Assert.assertThat;
> > +import static org.junit.Assume.assumeTrue;
> >
> > /**
> > - * Tests for the {@code org.apache.calcite.adapter.mongodb} package.
> > - *
> > - * <p>Before calling this test, you need to populate MongoDB, as
> follows:
> > + * Used to trigger integration tests from maven (thus class name is
> suffixed with {@code IT}).
> >  *
> > - * <blockquote><code>
> > - * git clone https://github.com/vlsi/calcite-test-dataset<br>
> > - * cd calcite-test-dataset<br>
> > - * mvn install
> > - * </code></blockquote>
> > + * <p>If you want to run integration tests from IDE manually set
> > + * {@code -Dcalcite.integrationTest=true} system property.
> >  *
> > - * <p>This will create a virtual machine with MongoDB and "zips" and
> "foodmart"
> > - * data sets.
> > + * For command line use:
> > + * <pre>
> > + *     $ mvn install -Pit
> > + * </pre>
> > + * </p>
> >  */
> > -public class MongoAdapterIT {
> > -  public static final String MONGO_FOODMART_SCHEMA = "     {\n"
> > -      + "       type: 'custom',\n"
> > -      + "       name: '_foodmart',\n"
> > -      + "       factory:
> 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> > -      + "       operand: {\n"
> > -      + "         host: 'localhost',\n"
> > -      + "         database: 'foodmart'\n"
> > -      + "       }\n"
> > -      + "     },\n"
> > -      + "     {\n"
> > -      + "       name: 'foodmart',\n"
> > -      + "       tables: [\n"
> > -      + "         {\n"
> > -      + "           name: 'sales_fact_1997',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double)
> AS \"product_id\" from \"_foodmart\".\"sales_fact_1997\"'\n"
> > -      + "         },\n"
> > -      + "         {\n"
> > -      + "           name: 'sales_fact_1998',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double)
> AS \"product_id\" from \"_foodmart\".\"sales_fact_1998\"'\n"
> > -      + "         },\n"
> > -      + "         {\n"
> > -      + "           name: 'store',\n"
> > -      + "           type: 'view',\n"
> > -      + "           sql: 'select cast(_MAP[\\'store_id\\'] AS double)
> AS \"store_id\", cast(_MAP[\\'store_name\\'] AS varchar(20)) AS
> \"store_name\" from \"_foodmart\".\"store\"'\n"
> > -      + "         },\n"
> > -      + "

Re: [2/2] calcite git commit: [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)

Posted by Julian Hyde <jh...@apache.org>.
This change introduces a javadoc error:

[ERROR] /home/jhyde/regress/calcite/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java:35: error: unexpected end tag: </p>
[ERROR]  * </p>

Please fix ASAP.


> On Jun 13, 2018, at 6:00 PM, mmior@apache.org wrote:
> 
> [CALCITE-2345] Running Unit tests with Fongo and integration tests with real mongo instance (Andrei Sereda)
> 
> Better test coverage for unit tests using Fongo in-memory implementation of Mongo API.
> New code will decide (at runtime) what connection to make: fongo vs mongo. Identical tests will be run against
> both databases (depending on maven profile surefire vs failsafe)
> 
> Close apache/calcite#723
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
> Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/dcf396a5
> Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/dcf396a5
> Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/dcf396a5
> 
> Branch: refs/heads/master
> Commit: dcf396a5ca92ee0714c28d9a3c310f53f46220ab
> Parents: f84a3eb
> Author: Andrei Sereda <an...@nospam.com>
> Authored: Tue Jun 5 16:23:56 2018 -0400
> Committer: Michael Mior <mm...@uwaterloo.ca>
> Committed: Wed Jun 13 18:59:15 2018 -0400
> 
> ----------------------------------------------------------------------
> mongodb/pom.xml                                 |  10 +-
> .../adapter/mongodb/MongoAdapterTest.java       | 794 +++++++++++++++---
> .../adapter/mongodb/MongoDatabaseRule.java      |  83 ++
> .../org/apache/calcite/test/MongoAdapterIT.java | 830 +------------------
> .../apache/calcite/test/MongoAssertions.java    | 101 +++
> .../test/resources/mongo-foodmart-model.json    | 221 -----
> mongodb/src/test/resources/mongo-model.json     |  70 ++
> .../src/test/resources/mongo-zips-model.json    |  41 -
> mongodb/src/test/resources/zips-mini.json       | 149 ++++
> pom.xml                                         |   9 +
> 10 files changed, 1132 insertions(+), 1176 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/pom.xml
> ----------------------------------------------------------------------
> diff --git a/mongodb/pom.xml b/mongodb/pom.xml
> index 25c0642..63cc147 100644
> --- a/mongodb/pom.xml
> +++ b/mongodb/pom.xml
> @@ -71,6 +71,11 @@ limitations under the License.
>       <scope>test</scope>
>     </dependency>
>     <dependency>
> +      <groupId>net.hydromatic</groupId>
> +      <artifactId>foodmart-data-json</artifactId>
> +      <scope>test</scope>
> +    </dependency>
> +    <dependency>
>       <groupId>org.mongodb</groupId>
>       <artifactId>mongo-java-driver</artifactId>
>     </dependency>
> @@ -78,11 +83,6 @@ limitations under the License.
>       <groupId>org.slf4j</groupId>
>       <artifactId>slf4j-api</artifactId>
>     </dependency>
> -    <dependency>
> -      <groupId>org.slf4j</groupId>
> -      <artifactId>slf4j-log4j12</artifactId>
> -      <scope>test</scope>
> -    </dependency>
>   </dependencies>
> 
>   <build>
> 
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> ----------------------------------------------------------------------
> diff --git a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> index b6ca4f3..a4061e4 100644
> --- a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> +++ b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoAdapterTest.java
> @@ -16,143 +16,749 @@
>  */
> package org.apache.calcite.adapter.mongodb;
> 
> -import org.apache.calcite.jdbc.CalciteConnection;
> +import org.apache.calcite.schema.Schema;
> +import org.apache.calcite.schema.SchemaFactory;
> import org.apache.calcite.schema.SchemaPlus;
> import org.apache.calcite.test.CalciteAssert;
> +import org.apache.calcite.test.MongoAssertions;
> 
> -import com.github.fakemongo.junit.FongoRule;
> +
> +import org.apache.calcite.util.Bug;
> +import org.apache.calcite.util.Util;
> +
> +import com.google.common.base.Function;
> +import com.google.common.base.Preconditions;
> +import com.google.common.io.LineProcessor;
> +import com.google.common.io.Resources;
> import com.mongodb.client.MongoCollection;
> import com.mongodb.client.MongoDatabase;
> 
> +import net.hydromatic.foodmart.data.json.FoodmartJson;
> +
> +import org.bson.BsonDateTime;
> import org.bson.BsonDocument;
> +import org.bson.BsonInt32;
> +import org.bson.BsonString;
> import org.bson.Document;
> 
> -import org.junit.Before;
> +import org.hamcrest.CoreMatchers;
> +
> +import org.junit.Assert;
> +import org.junit.BeforeClass;
> +import org.junit.ClassRule;
> import org.junit.Ignore;
> -import org.junit.Rule;
> import org.junit.Test;
> 
> -import java.sql.Connection;
> -import java.sql.DriverManager;
> +import java.io.IOException;
> +import java.io.UncheckedIOException;
> +import java.net.URL;
> +import java.nio.charset.StandardCharsets;
> +import java.sql.ResultSet;
> import java.sql.SQLException;
> +import java.text.SimpleDateFormat;
> +import java.util.Date;
> +import java.util.List;
> +import java.util.Locale;
> +import java.util.Map;
> 
> /**
> - * Tests current adapter using in-memory (fake) implementation of Mongo API:
> - * <a href="https://github.com/fakemongo/fongo">Fongo</a>.
> - *
> + * Testing mongo adapter functionality. By default runs with
> + * <a href="https://github.com/fakemongo/fongo">Fongo</a> unless {@code IT} maven profile is enabled
> + * (via {@code $ mvn -Pit install}).
>  */
> -public class MongoAdapterTest {
> +public class MongoAdapterTest implements SchemaFactory {
> +
> +  /** Connection factory based on the "mongo-zips" model. */
> +  protected static final URL MODEL = MongoAdapterTest.class.getResource("/mongo-model.json");
> +
> +  /** Number of records in local file */
> +  protected static final int ZIPS_SIZE = 149;
> +
> +  @ClassRule
> +  public static final MongoDatabaseRule RULE = MongoDatabaseRule.create();
> +
> +  private static MongoSchema schema;
> +
> +  @BeforeClass
> +  public static void setUp() throws Exception {
> +    MongoDatabase database = RULE.database();
> +
> +    populate(database.getCollection("zips"), MongoAdapterTest.class.getResource("/zips-mini.json"));
> +    populate(database.getCollection("store"), FoodmartJson.class.getResource("/store.json"));
> +    populate(database.getCollection("warehouse"),
> +            FoodmartJson.class.getResource("/warehouse.json"));
> 
> -  @Rule
> -  public final FongoRule rule = new FongoRule();
> +    // Manually insert data for data-time test.
> +    MongoCollection<BsonDocument> datatypes =  database.getCollection("datatypes")
> +            .withDocumentClass(BsonDocument.class);
> +    if (datatypes.count() > 0) {
> +      datatypes.deleteMany(new BsonDocument());
> +    }
> +    BsonDocument doc = new BsonDocument();
> +    Date date = new SimpleDateFormat("yyyy-MM-dd", Locale.getDefault()).parse("2012-09-05");
> +    doc.put("date", new BsonDateTime(date.getTime()));
> +    doc.put("value", new BsonInt32(1231));
> +    doc.put("ownerId", new BsonString("531e7789e4b0853ddb861313"));
> +    datatypes.insertOne(doc);
> 
> -  private MongoDatabase mongoDb;
> -  private MongoCollection<Document> zips;
> +    schema = new MongoSchema(database);
> +  }
> +
> +  private static void populate(MongoCollection<Document> collection, URL resource)
> +          throws IOException {
> +    Preconditions.checkNotNull(collection, "collection");
> +
> +    if (collection.count() > 0) {
> +      // delete any existing documents (run from a clean set)
> +      collection.deleteMany(new BsonDocument());
> +    }
> 
> -  @Before
> -  public void setUp() throws Exception {
> -    mongoDb = rule.getDatabase(getClass().getSimpleName());
> -    zips = mongoDb.getCollection("zips");
> +    MongoCollection<BsonDocument> bsonCollection = collection.withDocumentClass(BsonDocument.class);
> +    Resources.readLines(resource, StandardCharsets.UTF_8, new LineProcessor<Void>() {
> +      @Override public boolean processLine(String line) throws IOException {
> +        bsonCollection.insertOne(BsonDocument.parse(line));
> +        return true;
> +      }
> +
> +      @Override public Void getResult() {
> +        return null;
> +      }
> +    });
>   }
> 
>   /**
> -   * Handcrafted connection where we manually added {@link MongoSchema}
> +   *  Returns always the same schema to avoid initialization costs.
>    */
> -  private CalciteAssert.ConnectionFactory newConnectionFactory() {
> -    return new CalciteAssert.ConnectionFactory() {
> -      @Override public Connection createConnection() throws SQLException {
> -        Connection connection = DriverManager.getConnection("jdbc:calcite:");
> -        final SchemaPlus root = connection.unwrap(CalciteConnection.class).getRootSchema();
> -        root.add("mongo", new MongoSchema(mongoDb));
> -        return connection;
> -      }
> -    };
> +  @Override public Schema create(SchemaPlus parentSchema, String name,
> +                                 Map<String, Object> operand) {
> +    return schema;
> +  }
> +
> +  private CalciteAssert.AssertThat assertModel(String model) {
> +    // ensure that Schema from this instance is being used
> +    model = model.replace(MongoSchemaFactory.class.getName(), MongoAdapterTest.class.getName());
> +
> +    return CalciteAssert.that()
> +            .withModel(model);
> +  }
> +
> +  private CalciteAssert.AssertThat assertModel(URL url) {
> +    Preconditions.checkNotNull(url, "url");
> +    try {
> +      return assertModel(Resources.toString(url, StandardCharsets.UTF_8));
> +    } catch (IOException e) {
> +      throw new UncheckedIOException(e);
> +    }
>   }
> 
>   @Test
> -  public void single() {
> -    zips.insertOne(new Document());
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select * from \"mongo\".\"zips\"")
> +  public void testSort() {
> +    assertModel(MODEL)
> +            .query("select * from zips order by state")
> +            .returnsCount(ZIPS_SIZE)
> +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> +                    + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
> +                    + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> +                    + "      MongoTableScan(table=[[mongo_raw, zips]])");
> +  }
> +
> +  @Test public void testSortLimit() {
> +    assertModel(MODEL)
> +            .query("select state, id from zips\n"
> +                    + "order by state, id offset 2 rows fetch next 3 rows only")
> +            .returns("STATE=AK; ID=99801\n"
> +                    + "STATE=AL; ID=35215\n"
> +                    + "STATE=AL; ID=35401\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state', ID: '$_id'}}",
> +                            "{$sort: {STATE: 1, ID: 1}}",
> +                            "{$skip: 2}",
> +                            "{$limit: 3}"));
> +  }
> +
> +  @Test public void testOffsetLimit() {
> +    assertModel(MODEL)
> +            .query("select state, id from zips\n"
> +                    + "offset 2 fetch next 3 rows only")
>             .runs()
> -            .returnsCount(1);
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$skip: 2}",
> +                            "{$limit: 3}",
> +                            "{$project: {STATE: '$state', ID: '$_id'}}"));
>   }
> 
> -  @Test
> -  public void empty() {
> -    // for some reason fongo doesn't list collection if it was unused
> -    zips.insertOne(new Document());
> -    zips.deleteMany(new BsonDocument());
> -
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select * from \"mongo\".\"zips\"")
> +  @Test public void testLimit() {
> +    assertModel(MODEL)
> +            .query("select state, id from zips\n"
> +                    + "fetch next 3 rows only")
>             .runs()
> -            .returnsCount(0);
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$limit: 3}",
> +                            "{$project: {STATE: '$state', ID: '$_id'}}"));
>   }
> 
> -  @Test
> -  public void filter() {
> -    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
> -    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
> +  @Ignore
> +  @Test public void testFilterSort() {
> +    // LONGITUDE and LATITUDE are null because of CALCITE-194.
> +    Util.discard(Bug.CALCITE_194_FIXED);
> +    assertModel(MODEL)
> +            .query("select * from zips\n"
> +                    + "where city = 'SPRINGFIELD' and id >= '70000'\n"
> +                    + "order by state, id")
> +            .returns(""
> +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=752; STATE=AR; ID=72157\n"
> +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=1992; STATE=CO; ID=81073\n"
> +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=5597; STATE=LA; ID=70462\n"
> +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=32384; STATE=OR; ID=97477\n"
> +                    + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=27521; STATE=OR; ID=97478\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{\n"
> +                                    + "  $match: {\n"
> +                                    + "    city: \"SPRINGFIELD\",\n"
> +                                    + "    _id: {\n"
> +                                    + "      $gte: \"70000\"\n"
> +                                    + "    }\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
> +                            "{$sort: {STATE: 1, ID: 1}}"))
> +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> +                    + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
> +                    + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> +                    + "      MongoFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
> +                    + "        MongoTableScan(table=[[mongo_raw, zips]])");
> +  }
> 
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
> -                    + " where _MAP['STATE'] = 'NY'")
> -            .returns("city=New York\n");
> +  @Test public void testFilterSortDesc() {
> +    assertModel(MODEL)
> +            .query("select * from zips\n"
> +                    + "where pop BETWEEN 45000 AND 46000\n"
> +                    + "order by state desc, pop")
> +            .limit(4)
> +            .returns("CITY=BECKLEY; LONGITUDE=null; LATITUDE=null; POP=45196; STATE=WV; ID=25801\n"
> +                    + "CITY=ROCKERVILLE; LONGITUDE=null; LATITUDE=null; POP=45328; STATE=SD; ID=57701\n"
> +                    + "CITY=PAWTUCKET; LONGITUDE=null; LATITUDE=null; POP=45442; STATE=RI; ID=02860\n"
> +                    + "CITY=LAWTON; LONGITUDE=null; LATITUDE=null; POP=45542; STATE=OK; ID=73505\n");
> +  }
> 
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
> -                   + " where _MAP['STATE'] = 'DC'")
> -            .returns("city=Washington\n");
> +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> +  @Test public void testUnionPlan() {
> +    assertModel(MODEL)
> +            .query("select * from \"sales_fact_1997\"\n"
> +                    + "union all\n"
> +                    + "select * from \"sales_fact_1998\"")
> +            .explainContains("PLAN=EnumerableUnion(all=[true])\n"
> +                    + "  MongoToEnumerableConverter\n"
> +                    + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
> +                    + "      MongoTableScan(table=[[_foodmart, sales_fact_1997]])\n"
> +                    + "  MongoToEnumerableConverter\n"
> +                    + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
> +                    + "      MongoTableScan(table=[[_foodmart, sales_fact_1998]])")
> +            .limit(2)
> +            .returns(
> +                    MongoAssertions.checkResultUnordered(
> +                            "product_id=337", "product_id=1512"));
> +  }
> 
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
> -                    + " where _MAP['STATE'] in ('DC', 'NY')")
> -            .returns("city=New York\ncity=Washington\n");
> +  @Ignore(
> +          "java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Double")
> +  @Test public void testFilterUnionPlan() {
> +    assertModel(MODEL)
> +            .query("select * from (\n"
> +                    + "  select * from \"sales_fact_1997\"\n"
> +                    + "  union all\n"
> +                    + "  select * from \"sales_fact_1998\")\n"
> +                    + "where \"product_id\" = 1")
> +            .runs();
>   }
> 
> -  @Test
> -  public void limit() {
> -    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
> -    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
> +  /** Tests that we don't generate multiple constraints on the same column.
> +   * MongoDB doesn't like it. If there is an '=', it supersedes all other
> +   * operators. */
> +  @Test public void testFilterRedundant() {
> +    assertModel(MODEL)
> +            .query(
> +                    "select * from zips where state > 'CA' and state < 'AZ' and state = 'OK'")
> +            .runs()
> +            .queryContains(
> +                    mongoChecker(
> +                            "{\n"
> +                                    + "  \"$match\": {\n"
> +                                    + "    \"state\": \"OK\"\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}"));
> +  }
> +
> +  @Test public void testSelectWhere() {
> +    assertModel(MODEL)
> +            .query(
> +                    "select * from \"warehouse\" where \"warehouse_state_province\" = 'CA'")
> +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> +                    + "  MongoProject(warehouse_id=[CAST(ITEM($0, 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> +                    + "    MongoFilter(condition=[=(CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> +                    + "      MongoTableScan(table=[[mongo_raw, warehouse]])")
> +            .returns(
> +                    MongoAssertions.checkResultUnordered(
> +                            "warehouse_id=6; warehouse_state_province=CA",
> +                            "warehouse_id=7; warehouse_state_province=CA",
> +                            "warehouse_id=14; warehouse_state_province=CA",
> +                            "warehouse_id=24; warehouse_state_province=CA"))
> +            .queryContains(
> +                    // Per https://issues.apache.org/jira/browse/CALCITE-164,
> +                    // $match must occur before $project for good performance.
> +                    mongoChecker(
> +                            "{\n"
> +                                    + "  \"$match\": {\n"
> +                                    + "    \"warehouse_state_province\": \"CA\"\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {warehouse_id: 1, warehouse_state_province: 1}}"));
> +  }
> 
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select * from \"mongo\".\"zips\" limit 1")
> -            .returnsCount(1);
> +  @Test public void testInPlan() {
> +    assertModel(MODEL)
> +            .query("select \"store_id\", \"store_name\" from \"store\"\n"
> +                    + "where \"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
> +            .returns(
> +                    MongoAssertions.checkResultUnordered(
> +                            "store_id=1; store_name=Store 1",
> +                            "store_id=3; store_name=Store 3",
> +                            "store_id=7; store_name=Store 7",
> +                            "store_id=10; store_name=Store 10",
> +                            "store_id=11; store_name=Store 11",
> +                            "store_id=15; store_name=Store 15",
> +                            "store_id=16; store_name=Store 16",
> +                            "store_id=24; store_name=Store 24"))
> +            .queryContains(
> +                    mongoChecker(
> +                            "{\n"
> +                                    + "  \"$match\": {\n"
> +                                    + "    \"$or\": [\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 1\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 10\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 11\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 15\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 16\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 24\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 3\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"store_name\": \"Store 7\"\n"
> +                                    + "      }\n"
> +                                    + "    ]\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {store_id: 1, store_name: 1}}"));
> +  }
> 
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select * from \"mongo\".\"zips\" limit 2")
> -            .returnsCount(2);
> +  /** Simple query based on the "mongo-zips" model. */
> +  @Test public void testZips() {
> +    assertModel(MODEL)
> +            .query("select state, city from zips")
> +            .returnsCount(ZIPS_SIZE);
> +  }
> 
> +  @Test public void testCountGroupByEmpty() {
> +    assertModel(MODEL)
> +            .query("select count(*) from zips")
> +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n", ZIPS_SIZE))
> +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> +                    + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
> +                    + "    MongoTableScan(table=[[mongo_raw, zips]])")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$group: {_id: {}, 'EXPR$0': {$sum: 1}}}"));
>   }
> 
> -  /**
> -   * Following queries are not supported in Mongo adapter :
> -   * <pre>
> -   * {@code A and (B or C)}
> -   * {@code (A or B) and C}
> -   * </pre>
> +  @Test public void testCountGroupByEmptyMultiplyBy2() {
> +    // This operation is not supported by fongo: https://github.com/fakemongo/fongo/issues/152
> +    MongoAssertions.assumeRealMongoInstance();
> +
> +    assertModel(MODEL)
> +            .query("select count(*)*2 from zips")
> +            .returns(String.format(Locale.getDefault(), "EXPR$0=%d\n", ZIPS_SIZE * 2))
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$group: {_id: {}, _0: {$sum: 1}}}",
> +                            "{$project: {'EXPR$0': {$multiply: ['$_0', {$literal: 2}]}}}"));
> +  }
> +
> +  @Test public void testGroupByOneColumnNotProjected() {
> +    assertModel(MODEL)
> +            .query("select count(*) from zips group by state order by 1")
> +            .limit(2)
> +            .returns("EXPR$0=2\n"
> +                    + "EXPR$0=2\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', 'EXPR$0': {$sum: 1}}}",
> +                            "{$project: {STATE: '$_id', 'EXPR$0': '$EXPR$0'}}",
> +                            "{$project: {'EXPR$0': 1}}",
> +                            "{$sort: {EXPR$0: 1}}"));
> +  }
> +
> +  @Test public void testGroupByOneColumn() {
> +    assertModel(MODEL)
> +            .query(
> +                    "select state, count(*) as c from zips group by state order by state")
> +            .limit(2)
> +            .returns("STATE=AK; C=3\nSTATE=AL; C=3\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> +                            "{$project: {STATE: '$_id', C: '$C'}}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Test public void testGroupByOneColumnReversed() {
> +    // Note extra $project compared to testGroupByOneColumn.
> +    assertModel(MODEL)
> +            .query(
> +                    "select count(*) as c, state from zips group by state order by state")
> +            .limit(2)
> +            .returns("C=3; STATE=AK\nC=3; STATE=AL\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> +                            "{$project: {STATE: '$_id', C: '$C'}}",
> +                            "{$project: {C: 1, STATE: 1}}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Test public void testGroupByAvg() {
> +    assertModel(MODEL)
> +            .query(
> +                    "select state, avg(pop) as a from zips group by state order by state")
> +            .limit(2)
> +            .returns("STATE=AK; A=26856\nSTATE=AL; A=43383\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {POP: '$pop', STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', A: {$avg: '$POP'}}}",
> +                            "{$project: {STATE: '$_id', A: '$A'}}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Test public void testGroupByAvgSumCount() {
> +    // This operation not supported by fongo: https://github.com/fakemongo/fongo/issues/152
> +    MongoAssertions.assumeRealMongoInstance();
> +    assertModel(MODEL)
> +            .query(
> +                    "select state, avg(pop) as a, sum(pop) as s, count(pop) as c from zips group by state order by state")
> +            .limit(2)
> +            .returns("STATE=AK; A=26856; S=80568; C=3\n"
> +                    + "STATE=AL; A=43383; S=130151; C=3\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {POP: '$pop', STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', _1: {$sum: '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
> +                            "{$project: {STATE: '$_id', _1: '$_1', _2: '$_2'}}",
> +                            "{$sort: {STATE: 1}}",
> +                            "{$project: {STATE: 1, A: {$divide: [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S: {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
> +  }
> +
> +  @Test public void testGroupByHaving() {
> +    assertModel(MODEL)
> +            .query("select state, count(*) as c from zips\n"
> +                    + "group by state having count(*) > 2 order by state")
> +            .returnsCount(47)
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> +                            "{$project: {STATE: '$_id', C: '$C'}}",
> +                            "{\n"
> +                                    + "  \"$match\": {\n"
> +                                    + "    \"C\": {\n"
> +                                    + "      \"$gt\": 2\n"
> +                                    + "    }\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
> +  @Test public void testGroupByHaving2() {
> +    assertModel(MODEL)
> +            .query("select state, count(*) as c from zips\n"
> +                    + "group by state having sum(pop) > 12000000")
> +            .returns("STATE=NY; C=1596\n"
> +                    + "STATE=TX; C=1676\n"
> +                    + "STATE=FL; C=826\n"
> +                    + "STATE=CA; C=1523\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {STATE: '$state', POP: '$pop'}}",
> +                            "{$group: {_id: '$STATE', C: {$sum: 1}, _2: {$sum: '$POP'}}}",
> +                            "{$project: {STATE: '$_id', C: '$C', _2: '$_2'}}",
> +                            "{\n"
> +                                    + "  $match: {\n"
> +                                    + "    _2: {\n"
> +                                    + "      $gt: 12000000\n"
> +                                    + "    }\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {STATE: 1, C: 1}}"));
> +  }
> +
> +  @Test public void testGroupByMinMaxSum() {
> +    assertModel(MODEL)
> +            .query("select count(*) as c, state,\n"
> +                    + " min(pop) as min_pop, max(pop) as max_pop, sum(pop) as sum_pop\n"
> +                    + "from zips group by state order by state")
> +            .limit(2)
> +            .returns("C=3; STATE=AK; MIN_POP=23238; MAX_POP=32383; SUM_POP=80568\n"
> +                    + "C=3; STATE=AL; MIN_POP=42124; MAX_POP=44165; SUM_POP=130151\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {POP: '$pop', STATE: '$state'}}",
> +                            "{$group: {_id: '$STATE', C: {$sum: 1}, MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum: '$POP'}}}",
> +                            "{$project: {STATE: '$_id', C: '$C', MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
> +                            "{$project: {C: 1, STATE: 1, MIN_POP: 1, MAX_POP: 1, SUM_POP: 1}}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Test public void testGroupComposite() {
> +    assertModel(MODEL)
> +            .query("select count(*) as c, state, city from zips\n"
> +                    + "group by state, city\n"
> +                    + "order by c desc, city\n"
> +                    + "limit 2")
> +            .returns("C=1; STATE=SD; CITY=ABERDEEN\n"
> +                      + "C=1; STATE=SC; CITY=AIKEN\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {CITY: '$city', STATE: '$state'}}",
> +                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}, C: {$sum: 1}}}",
> +                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE', C: '$C'}}",
> +                            "{$sort: {C: -1, CITY: 1}}",
> +                            "{$limit: 2}",
> +                            "{$project: {C: 1, STATE: 1, CITY: 1}}"));
> +  }
> +
> +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> +  @Test public void testDistinctCount() {
> +    assertModel(MODEL)
> +            .query("select state, count(distinct city) as cdc from zips\n"
> +                    + "where state in ('CA', 'TX') group by state order by state")
> +            .returns("STATE=CA; CDC=1072\n"
> +                    + "STATE=TX; CDC=1233\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{\n"
> +                                    + "  \"$match\": {\n"
> +                                    + "    \"$or\": [\n"
> +                                    + "      {\n"
> +                                    + "        \"state\": \"CA\"\n"
> +                                    + "      },\n"
> +                                    + "      {\n"
> +                                    + "        \"state\": \"TX\"\n"
> +                                    + "      }\n"
> +                                    + "    ]\n"
> +                                    + "  }\n"
> +                                    + "}",
> +                            "{$project: {CITY: '$city', STATE: '$state'}}",
> +                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
> +                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
> +                            "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> +                            "{$sort: {STATE: 1}}"));
> +  }
> +
> +  @Test public void testDistinctCountOrderBy() {
> +    // java.lang.ClassCastException: com.mongodb.BasicDBObject cannot be cast to java.lang.Number
> +    // https://github.com/fakemongo/fongo/issues/152
> +    MongoAssertions.assumeRealMongoInstance();
> +    assertModel(MODEL)
> +            .query("select state, count(distinct city) as cdc\n"
> +                    + "from zips\n"
> +                    + "group by state\n"
> +                    + "order by cdc desc limit 5")
> +            .returns("STATE=VA; CDC=3\n"
> +                    + "STATE=NY; CDC=3\n"
> +                    + "STATE=SC; CDC=3\n"
> +                    + "STATE=RI; CDC=3\n"
> +                    + "STATE=WV; CDC=3\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {CITY: '$city', STATE: '$state'}}",
> +                            "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
> +                            "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
> +                            "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> +                            "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> +                            "{$sort: {CDC: -1}}",
> +                            "{$limit: 5}"));
> +  }
> +
> +  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> +  @Test public void testProject() {
> +    assertModel(MODEL)
> +            .query("select state, city, 0 as zero from zips order by state, city")
> +            .limit(2)
> +            .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
> +                    + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
> +            .queryContains(
> +                    mongoChecker(
> +                            "{$project: {CITY: '$city', STATE: '$state'}}",
> +                            "{$sort: {STATE: 1, CITY: 1}}",
> +                            "{$project: {STATE: 1, CITY: 1, ZERO: {$literal: 0}}}"));
> +  }
> +
> +  @Test public void testFilter() {
> +    assertModel(MODEL)
> +            .query("select state, city from zips where state = 'CA'")
> +            .limit(2)
> +            .returns("STATE=CA; CITY=LOS ANGELES\n"
> +                    + "STATE=CA; CITY=BELL GARDENS\n")
> +            .explainContains("PLAN=MongoToEnumerableConverter\n"
> +                    + "  MongoProject(STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> +                    + "    MongoFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> +                    + "      MongoTableScan(table=[[mongo_raw, zips]])");
> +  }
> +
> +  /** MongoDB's predicates are handed (they can only accept literals on the
> +   * right-hand size) so it's worth testing that we handle them right both
> +   * ways around. */
> +  @Test public void testFilterReversed() {
> +    assertModel(MODEL)
> +            .query("select state, city from zips where 'WI' < state")
> +            .limit(2)
> +            .returns("STATE=WV; CITY=BECKLEY\nSTATE=WV; CITY=ELM GROVE\n");
> +
> +    assertModel(MODEL)
> +            .query("select state, city from zips where state > 'WI'")
> +            .limit(2)
> +            .returns("STATE=WV; CITY=BECKLEY\n"
> +                    + "STATE=WV; CITY=ELM GROVE\n");
> +  }
> +
> +  /** MongoDB's predicates are handed (they can only accept literals on the
> +   * right-hand size) so it's worth testing that we handle them right both
> +   * ways around.
>    *
> -   * @see <a href="https://issues.apache.org/jira/browse/CALCITE-2331">[CALCITE-2331]</a>
> -   */
> -  @Ignore("broken; [CALCITE-2331] is logged to fix it")
> -  @Test
> -  public void validateCALCITE2331() {
> -    zips.insertOne(new Document("CITY", "New York").append("STATE", "NY"));
> -    zips.insertOne(new Document("CITY", "Washington").append("STATE", "DC"));
> -
> -    CalciteAssert.that()
> -            .with(newConnectionFactory())
> -            .query("select cast(_MAP['CITY'] as varchar(20)) as \"city\" from \"mongo\".\"zips\" "
> -                    + " where _MAP['STATE'] in ('DC', 'NY') and _MAP['CITY'] = 'New York'")
> -            .returns("city=New York\n");
> +   * <p>Test case for
> +   * <a href="https://issues.apache.org/jira/browse/CALCITE-740">[CALCITE-740]
> +   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>. */
> +  @Test public void testFilterPair() {
> +    final int gt9k = 148;
> +    final int lt9k = 1;
> +    final int gt8k = 148;
> +    final int lt8k = 1;
> +    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
> +    checkPredicate(gt9k, "where pop > 9000");
> +    checkPredicate(lt9k, "where pop < 9000");
> +    checkPredicate(gt8k, "where pop > 8000");
> +    checkPredicate(lt8k, "where pop < 8000");
> +    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
> +    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
> +    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
> +    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
> +  }
> +
> +  private void checkPredicate(int expected, String q) {
> +    assertModel(MODEL)
> +            .query("select count(*) as c from zips\n"
> +                    + q)
> +            .returns("C=" + expected + "\n");
> +    assertModel(MODEL)
> +            .query("select * from zips\n"
> +                    + q)
> +            .returnsCount(expected);
> +  }
> +
> +  /** Test case for
> +   * <a href="https://issues.apache.org/jira/browse/CALCITE-286">[CALCITE-286]
> +   * Error casting MongoDB date</a>. */
> +  @Test public void testDate() {
> +    // Assumes that you have created the following collection before running
> +    // this test:
> +    //
> +    // $ mongo
> +    // > use test
> +    // switched to db test
> +    // > db.createCollection("datatypes")
> +    // { "ok" : 1 }
> +    // > db.datatypes.insert( {
> +    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
> +    //     "_class" : "com.ericblue.Test",
> +    //     "date" : ISODate("2012-09-05T07:00:00Z"),
> +    //     "value" : 1231,
> +    //     "ownerId" : "531e7789e4b0853ddb861313"
> +    //   } )
> +    assertModel("{\n"
> +            + "  version: '1.0',\n"
> +            + "  defaultSchema: 'test',\n"
> +            + "   schemas: [\n"
> +            + "     {\n"
> +            + "       type: 'custom',\n"
> +            + "       name: 'test',\n"
> +            + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> +            + "       operand: {\n"
> +            + "         host: 'localhost',\n"
> +            + "         database: 'test'\n"
> +            + "       }\n"
> +            + "     }\n"
> +            + "   ]\n"
> +            + "}")
> +            .query("select cast(_MAP['date'] as DATE) from \"datatypes\"")
> +            .returnsUnordered("EXPR$0=2012-09-05");
> +  }
> +
> +  /** Test case for
> +   * <a href="https://issues.apache.org/jira/browse/CALCITE-665">[CALCITE-665]
> +   * ClassCastException in MongoDB adapter</a>. */
> +  @Test public void testCountViaInt() {
> +    assertModel(MODEL)
> +        .query("select count(*) from zips")
> +        .returns(
> +            new Function<ResultSet, Void>() {
> +              public Void apply(ResultSet input) {
> +                try {
> +                  Assert.assertThat(input.next(), CoreMatchers.is(true));
> +                  Assert.assertThat(input.getInt(1), CoreMatchers.is(ZIPS_SIZE));
> +                  return null;
> +                } catch (SQLException e) {
> +                  throw new RuntimeException(e);
> +                }
> +              }
> +            });
> +  }
> +
> +  /** Returns a function that checks that a particular MongoDB pipeline is
> +   * generated to implement a query. */
> +  private static Function<List, Void> mongoChecker(final String... strings) {
> +    return new Function<List, Void>() {
> +      public Void apply(List actual) {
> +        Object[] actualArray =
> +                actual == null || actual.isEmpty()
> +                        ? null
> +                        : ((List) actual.get(0)).toArray();
> +        CalciteAssert.assertArrayEqual("expected MongoDB query not found",
> +                strings, actualArray);
> +        return null;
> +      }
> +    };
>   }
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> ----------------------------------------------------------------------
> diff --git a/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> new file mode 100644
> index 0000000..90bd759
> --- /dev/null
> +++ b/mongodb/src/test/java/org/apache/calcite/adapter/mongodb/MongoDatabaseRule.java
> @@ -0,0 +1,83 @@
> +/*
> + * 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.calcite.adapter.mongodb;
> +
> +import org.apache.calcite.test.MongoAssertions;
> +
> +import com.github.fakemongo.Fongo;
> +
> +import com.google.common.base.Preconditions;
> +
> +import com.mongodb.MongoClient;
> +import com.mongodb.client.MongoDatabase;
> +
> +import org.junit.rules.ExternalResource;
> +
> +/**
> + * Instantiates new connection to fongo (or mongo) database depending on current profile
> + * (unit or integration tests).
> + *
> + * By default, this rule is executed as part of a unit test and in-memory database
> + * <a href="https://github.com/fakemongo/fongo">fongo</a> is used.
> + *
> + * <p>However, if maven profile is set to {@code IT} (eg. via command line
> + * {@code $ mvn -Pit install}) this rule will connect to existing (external)
> + * mongo instance ({@code localhost})</p>
> + *
> + */
> +class MongoDatabaseRule extends ExternalResource {
> +
> +  private static final String DB_NAME = "test";
> +
> +  private final MongoDatabase database;
> +  private final MongoClient client;
> +
> +  private MongoDatabaseRule(MongoClient client) {
> +    this.client = Preconditions.checkNotNull(client, "client");
> +    this.database = client.getDatabase(DB_NAME);
> +  }
> +
> +  /**
> +   * Create an instance based on current maven profile (as defined by {@code -Pit}).
> +   */
> +  static MongoDatabaseRule create() {
> +    final MongoClient client;
> +    if (MongoAssertions.useMongo()) {
> +      // use to real client (connects to mongo)
> +      client = new MongoClient();
> +    } else if (MongoAssertions.useFongo()) {
> +      // in-memory DB (fake Mongo)
> +      client = new Fongo(MongoDatabaseRule.class.getSimpleName()).getMongo();
> +    } else {
> +      throw new UnsupportedOperationException("I can only connect to Mongo or Fongo instances");
> +    }
> +
> +    return new MongoDatabaseRule(client);
> +  }
> +
> +
> +  MongoDatabase database() {
> +    return database;
> +  }
> +
> +  @Override protected void after() {
> +    client.close();
> +  }
> +
> +}
> +
> +// End MongoDatabaseRule.java
> 
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> ----------------------------------------------------------------------
> diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> index 00ed03a..bda1163 100644
> --- a/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> +++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAdapterIT.java
> @@ -16,829 +16,29 @@
>  */
> package org.apache.calcite.test;
> 
> -import org.apache.calcite.linq4j.Ord;
> -import org.apache.calcite.util.Bug;
> -import org.apache.calcite.util.Pair;
> -import org.apache.calcite.util.Util;
> +import org.apache.calcite.adapter.mongodb.MongoAdapterTest;
> 
> -import com.google.common.base.Function;
> -import com.google.common.collect.ImmutableMap;
> -import com.google.common.collect.Lists;
> -import com.google.common.collect.Ordering;
> +import org.junit.BeforeClass;
> 
> -import org.hamcrest.CoreMatchers;
> -import org.junit.Ignore;
> -import org.junit.Test;
> -
> -import java.sql.ResultSet;
> -import java.sql.SQLException;
> -import java.util.Arrays;
> -import java.util.Collections;
> -import java.util.List;
> -
> -import static org.hamcrest.CoreMatchers.equalTo;
> -import static org.junit.Assert.assertThat;
> +import static org.junit.Assume.assumeTrue;
> 
> /**
> - * Tests for the {@code org.apache.calcite.adapter.mongodb} package.
> - *
> - * <p>Before calling this test, you need to populate MongoDB, as follows:
> + * Used to trigger integration tests from maven (thus class name is suffixed with {@code IT}).
>  *
> - * <blockquote><code>
> - * git clone https://github.com/vlsi/calcite-test-dataset<br>
> - * cd calcite-test-dataset<br>
> - * mvn install
> - * </code></blockquote>
> + * <p>If you want to run integration tests from IDE manually set
> + * {@code -Dcalcite.integrationTest=true} system property.
>  *
> - * <p>This will create a virtual machine with MongoDB and "zips" and "foodmart"
> - * data sets.
> + * For command line use:
> + * <pre>
> + *     $ mvn install -Pit
> + * </pre>
> + * </p>
>  */
> -public class MongoAdapterIT {
> -  public static final String MONGO_FOODMART_SCHEMA = "     {\n"
> -      + "       type: 'custom',\n"
> -      + "       name: '_foodmart',\n"
> -      + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> -      + "       operand: {\n"
> -      + "         host: 'localhost',\n"
> -      + "         database: 'foodmart'\n"
> -      + "       }\n"
> -      + "     },\n"
> -      + "     {\n"
> -      + "       name: 'foodmart',\n"
> -      + "       tables: [\n"
> -      + "         {\n"
> -      + "           name: 'sales_fact_1997',\n"
> -      + "           type: 'view',\n"
> -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double) AS \"product_id\" from \"_foodmart\".\"sales_fact_1997\"'\n"
> -      + "         },\n"
> -      + "         {\n"
> -      + "           name: 'sales_fact_1998',\n"
> -      + "           type: 'view',\n"
> -      + "           sql: 'select cast(_MAP[\\'product_id\\'] AS double) AS \"product_id\" from \"_foodmart\".\"sales_fact_1998\"'\n"
> -      + "         },\n"
> -      + "         {\n"
> -      + "           name: 'store',\n"
> -      + "           type: 'view',\n"
> -      + "           sql: 'select cast(_MAP[\\'store_id\\'] AS double) AS \"store_id\", cast(_MAP[\\'store_name\\'] AS varchar(20)) AS \"store_name\" from \"_foodmart\".\"store\"'\n"
> -      + "         },\n"
> -      + "         {\n"
> -      + "           name: 'warehouse',\n"
> -      + "           type: 'view',\n"
> -      + "           sql: 'select cast(_MAP[\\'warehouse_id\\'] AS double) AS \"warehouse_id\", cast(_MAP[\\'warehouse_state_province\\'] AS varchar(20)) AS \"warehouse_state_province\" from \"_foodmart\".\"warehouse\"'\n"
> -      + "         }\n"
> -      + "       ]\n"
> -      + "     }\n";
> -
> -  public static final String MONGO_FOODMART_MODEL = "{\n"
> -      + "  version: '1.0',\n"
> -      + "  defaultSchema: 'foodmart',\n"
> -      + "   schemas: [\n"
> -      + MONGO_FOODMART_SCHEMA
> -      + "   ]\n"
> -      + "}";
> -
> -  /** Connection factory based on the "mongo-zips" model. */
> -  public static final ImmutableMap<String, String> ZIPS =
> -      ImmutableMap.of("model",
> -          MongoAdapterIT.class.getResource("/mongo-zips-model.json")
> -              .getPath());
> -
> -  /** Connection factory based on the "mongo-zips" model. */
> -  public static final ImmutableMap<String, String> FOODMART =
> -      ImmutableMap.of("model",
> -          MongoAdapterIT.class.getResource("/mongo-foodmart-model.json")
> -              .getPath());
> -
> -  /** Whether to run Mongo tests. Enabled by default, however test is only
> -   * included if "it" profile is activated ({@code -Pit}). To disable,
> -   * specify {@code -Dcalcite.test.mongodb=false} on the Java command line. */
> -  public static final boolean ENABLED =
> -      Util.getBooleanProperty("calcite.test.mongodb", true);
> -
> -  /** Whether to run this test. */
> -  protected boolean enabled() {
> -    return ENABLED;
> -  }
> -
> -  /** Returns a function that checks that a particular MongoDB pipeline is
> -   * generated to implement a query. */
> -  private static Function<List, Void> mongoChecker(final String... strings) {
> -    return new Function<List, Void>() {
> -      public Void apply(List actual) {
> -        Object[] actualArray =
> -            actual == null || actual.isEmpty()
> -                ? null
> -                : ((List) actual.get(0)).toArray();
> -        CalciteAssert.assertArrayEqual("expected MongoDB query not found",
> -            strings, actualArray);
> -        return null;
> -      }
> -    };
> -  }
> -
> -  /** Similar to {@link CalciteAssert#checkResultUnordered}, but filters strings
> -   * before comparing them. */
> -  static Function<ResultSet, Void> checkResultUnordered(
> -      final String... lines) {
> -    return new Function<ResultSet, Void>() {
> -      public Void apply(ResultSet resultSet) {
> -        try {
> -          final List<String> expectedList =
> -              Ordering.natural().immutableSortedCopy(Arrays.asList(lines));
> -
> -          final List<String> actualList = Lists.newArrayList();
> -          CalciteAssert.toStringList(resultSet, actualList);
> -          for (int i = 0; i < actualList.size(); i++) {
> -            String s = actualList.get(i);
> -            actualList.set(i,
> -                s.replaceAll("\\.0;", ";").replaceAll("\\.0$", ""));
> -          }
> -          Collections.sort(actualList);
> -
> -          assertThat(Ordering.natural().immutableSortedCopy(actualList),
> -              equalTo(expectedList));
> -          return null;
> -        } catch (SQLException e) {
> -          throw new RuntimeException(e);
> -        }
> -      }
> -    };
> -  }
> -
> -  @Test public void testSort() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select * from zips order by state")
> -        .returnsCount(29353)
> -        .explainContains("PLAN=MongoToEnumerableConverter\n"
> -            + "  MongoSort(sort0=[$4], dir0=[ASC])\n"
> -            + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> -            + "      MongoTableScan(table=[[mongo_raw, zips]])");
> -  }
> -
> -  @Test public void testSortLimit() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, id from zips\n"
> -            + "order by state, id offset 2 rows fetch next 3 rows only")
> -        .returns("STATE=AK; ID=99503\n"
> -            + "STATE=AK; ID=99504\n"
> -            + "STATE=AK; ID=99505\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state', ID: '$_id'}}",
> -                "{$sort: {STATE: 1, ID: 1}}",
> -                "{$skip: 2}",
> -                "{$limit: 3}"));
> -  }
> -
> -  @Test public void testOffsetLimit() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, id from zips\n"
> -            + "offset 2 fetch next 3 rows only")
> -        .runs()
> -        .queryContains(
> -            mongoChecker(
> -                "{$skip: 2}",
> -                "{$limit: 3}",
> -                "{$project: {STATE: '$state', ID: '$_id'}}"));
> -  }
> -
> -  @Test public void testLimit() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, id from zips\n"
> -            + "fetch next 3 rows only")
> -        .runs()
> -        .queryContains(
> -            mongoChecker(
> -                "{$limit: 3}",
> -                "{$project: {STATE: '$state', ID: '$_id'}}"));
> -  }
> -
> -  @Ignore
> -  @Test public void testFilterSort() {
> -    // LONGITUDE and LATITUDE are null because of CALCITE-194.
> -    Util.discard(Bug.CALCITE_194_FIXED);
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select * from zips\n"
> -            + "where city = 'SPRINGFIELD' and id >= '70000'\n"
> -            + "order by state, id")
> -        .returns(""
> -            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=752; STATE=AR; ID=72157\n"
> -            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=1992; STATE=CO; ID=81073\n"
> -            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=5597; STATE=LA; ID=70462\n"
> -            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=32384; STATE=OR; ID=97477\n"
> -            + "CITY=SPRINGFIELD; LONGITUDE=null; LATITUDE=null; POP=27521; STATE=OR; ID=97478\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{\n"
> -                    + "  $match: {\n"
> -                    + "    city: \"SPRINGFIELD\",\n"
> -                    + "    _id: {\n"
> -                    + "      $gte: \"70000\"\n"
> -                    + "    }\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}",
> -                "{$sort: {STATE: 1, ID: 1}}"))
> -        .explainContains("PLAN=MongoToEnumerableConverter\n"
> -            + "  MongoSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
> -            + "    MongoProject(CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], LONGITUDE=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], LATITUDE=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], POP=[CAST(ITEM($0, 'pop')):INTEGER], STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], ID=[CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> -            + "      MongoFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, '_id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
> -            + "        MongoTableScan(table=[[mongo_raw, zips]])");
> -  }
> -
> -  @Test public void testFilterSortDesc() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select * from zips\n"
> -            + "where pop BETWEEN 20000 AND 20100\n"
> -            + "order by state desc, pop")
> -        .limit(4)
> -        .returns(""
> -            + "CITY=SHERIDAN; LONGITUDE=null; LATITUDE=null; POP=20025; STATE=WY; ID=82801\n"
> -            + "CITY=MOUNTLAKE TERRAC; LONGITUDE=null; LATITUDE=null; POP=20059; STATE=WA; ID=98043\n"
> -            + "CITY=FALMOUTH; LONGITUDE=null; LATITUDE=null; POP=20039; STATE=VA; ID=22405\n"
> -            + "CITY=FORT WORTH; LONGITUDE=null; LATITUDE=null; POP=20012; STATE=TX; ID=76104\n");
> -  }
> -
> -  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> -  @Test public void testUnionPlan() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .withModel(MONGO_FOODMART_MODEL)
> -        .query("select * from \"sales_fact_1997\"\n"
> -            + "union all\n"
> -            + "select * from \"sales_fact_1998\"")
> -        .explainContains("PLAN=EnumerableUnion(all=[true])\n"
> -            + "  MongoToEnumerableConverter\n"
> -            + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
> -            + "      MongoTableScan(table=[[_foodmart, sales_fact_1997]])\n"
> -            + "  MongoToEnumerableConverter\n"
> -            + "    MongoProject(product_id=[CAST(ITEM($0, 'product_id')):DOUBLE])\n"
> -            + "      MongoTableScan(table=[[_foodmart, sales_fact_1998]])")
> -        .limit(2)
> -        .returns(
> -            checkResultUnordered(
> -                "product_id=337", "product_id=1512"));
> -  }
> -
> -  @Ignore(
> -      "java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Double")
> -  @Test public void testFilterUnionPlan() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .withModel(MONGO_FOODMART_MODEL)
> -        .query("select * from (\n"
> -            + "  select * from \"sales_fact_1997\"\n"
> -            + "  union all\n"
> -            + "  select * from \"sales_fact_1998\")\n"
> -            + "where \"product_id\" = 1")
> -        .runs();
> -  }
> -
> -  /** Tests that we don't generate multiple constraints on the same column.
> -   * MongoDB doesn't like it. If there is an '=', it supersedes all other
> -   * operators. */
> -  @Test public void testFilterRedundant() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query(
> -            "select * from zips where state > 'CA' and state < 'AZ' and state = 'OK'")
> -        .runs()
> -        .queryContains(
> -            mongoChecker(
> -                "{\n"
> -                    + "  \"$match\": {\n"
> -                    + "    \"state\": \"OK\"\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {CITY: '$city', LONGITUDE: '$loc[0]', LATITUDE: '$loc[1]', POP: '$pop', STATE: '$state', ID: '$_id'}}"));
> -  }
> -
> -  @Test public void testSelectWhere() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .withModel(MONGO_FOODMART_MODEL)
> -        .query(
> -            "select * from \"warehouse\" where \"warehouse_state_province\" = 'CA'")
> -        .explainContains("PLAN=MongoToEnumerableConverter\n"
> -            + "  MongoProject(warehouse_id=[CAST(ITEM($0, 'warehouse_id')):DOUBLE], warehouse_state_province=[CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> -            + "    MongoFilter(condition=[=(CAST(ITEM($0, 'warehouse_state_province')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> -            + "      MongoTableScan(table=[[_foodmart, warehouse]])")
> -        .returns(
> -            checkResultUnordered(
> -                "warehouse_id=6; warehouse_state_province=CA",
> -                "warehouse_id=7; warehouse_state_province=CA",
> -                "warehouse_id=14; warehouse_state_province=CA",
> -                "warehouse_id=24; warehouse_state_province=CA"))
> -        .queryContains(
> -            // Per https://issues.apache.org/jira/browse/CALCITE-164,
> -            // $match must occur before $project for good performance.
> -            mongoChecker(
> -                "{\n"
> -                    + "  \"$match\": {\n"
> -                    + "    \"warehouse_state_province\": \"CA\"\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {warehouse_id: 1, warehouse_state_province: 1}}"));
> -  }
> -
> -  @Test public void testInPlan() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .withModel(MONGO_FOODMART_MODEL)
> -        .query("select \"store_id\", \"store_name\" from \"store\"\n"
> -            + "where \"store_name\" in ('Store 1', 'Store 10', 'Store 11', 'Store 15', 'Store 16', 'Store 24', 'Store 3', 'Store 7')")
> -        .returns(
> -            checkResultUnordered(
> -                "store_id=1; store_name=Store 1",
> -                "store_id=3; store_name=Store 3",
> -                "store_id=7; store_name=Store 7",
> -                "store_id=10; store_name=Store 10",
> -                "store_id=11; store_name=Store 11",
> -                "store_id=15; store_name=Store 15",
> -                "store_id=16; store_name=Store 16",
> -                "store_id=24; store_name=Store 24"))
> -        .queryContains(
> -            mongoChecker(
> -                "{\n"
> -                    + "  \"$match\": {\n"
> -                    + "    \"$or\": [\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 1\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 10\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 11\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 15\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 16\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 24\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 3\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"store_name\": \"Store 7\"\n"
> -                    + "      }\n"
> -                    + "    ]\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {store_id: 1, store_name: 1}}"));
> -  }
> -
> -  /** Simple query based on the "mongo-zips" model. */
> -  @Test public void testZips() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, city from zips")
> -        .returnsCount(29353);
> -  }
> -
> -  @Test public void testCountGroupByEmpty() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) from zips")
> -        .returns("EXPR$0=29353\n")
> -        .explainContains("PLAN=MongoToEnumerableConverter\n"
> -            + "  MongoAggregate(group=[{}], EXPR$0=[COUNT()])\n"
> -            + "    MongoTableScan(table=[[mongo_raw, zips]])")
> -        .queryContains(
> -            mongoChecker(
> -                "{$group: {_id: {}, 'EXPR$0': {$sum: 1}}}"));
> -  }
> -
> -  @Test public void testCountGroupByEmptyMultiplyBy2() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*)*2 from zips")
> -        .returns("EXPR$0=58706\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$group: {_id: {}, _0: {$sum: 1}}}",
> -                "{$project: {'EXPR$0': {$multiply: ['$_0', {$literal: 2}]}}}"));
> -  }
> -
> -  @Test public void testGroupByOneColumnNotProjected() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) from zips group by state order by 1")
> -        .limit(2)
> -        .returns("EXPR$0=24\n"
> -            + "EXPR$0=53\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', 'EXPR$0': {$sum: 1}}}",
> -                "{$project: {STATE: '$_id', 'EXPR$0': '$EXPR$0'}}",
> -                "{$project: {'EXPR$0': 1}}",
> -                "{$sort: {EXPR$0: 1}}"));
> -  }
> -
> -  @Test public void testGroupByOneColumn() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query(
> -            "select state, count(*) as c from zips group by state order by state")
> -        .limit(2)
> -        .returns("STATE=AK; C=195\n"
> -            + "STATE=AL; C=567\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> -                "{$project: {STATE: '$_id', C: '$C'}}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Test public void testGroupByOneColumnReversed() {
> -    // Note extra $project compared to testGroupByOneColumn.
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query(
> -            "select count(*) as c, state from zips group by state order by state")
> -        .limit(2)
> -        .returns("C=195; STATE=AK\n"
> -            + "C=567; STATE=AL\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> -                "{$project: {STATE: '$_id', C: '$C'}}",
> -                "{$project: {C: 1, STATE: 1}}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Test public void testGroupByAvg() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query(
> -            "select state, avg(pop) as a from zips group by state order by state")
> -        .limit(2)
> -        .returns("STATE=AK; A=2793\n"
> -            + "STATE=AL; A=7126\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {POP: '$pop', STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', A: {$avg: '$POP'}}}",
> -                "{$project: {STATE: '$_id', A: '$A'}}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Test public void testGroupByAvgSumCount() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query(
> -            "select state, avg(pop) as a, sum(pop) as s, count(pop) as c from zips group by state order by state")
> -        .limit(2)
> -        .returns("STATE=AK; A=2793; S=544698; C=195\n"
> -            + "STATE=AL; A=7126; S=4040587; C=567\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {POP: '$pop', STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', _1: {$sum: '$POP'}, _2: {$sum: {$cond: [ {$eq: ['POP', null]}, 0, 1]}}}}",
> -                "{$project: {STATE: '$_id', _1: '$_1', _2: '$_2'}}",
> -                "{$sort: {STATE: 1}}",
> -                "{$project: {STATE: 1, A: {$divide: [{$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, '$_2']}, S: {$cond:[{$eq: ['$_2', {$literal: 0}]},null,'$_1']}, C: '$_2'}}"));
> -  }
> -
> -  @Test public void testGroupByHaving() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, count(*) as c from zips\n"
> -            + "group by state having count(*) > 1500 order by state")
> -        .returns("STATE=CA; C=1516\n"
> -            + "STATE=NY; C=1595\n"
> -            + "STATE=TX; C=1671\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', C: {$sum: 1}}}",
> -                "{$project: {STATE: '$_id', C: '$C'}}",
> -                "{\n"
> -                    + "  \"$match\": {\n"
> -                    + "    \"C\": {\n"
> -                    + "      \"$gt\": 1500\n"
> -                    + "    }\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Ignore("https://issues.apache.org/jira/browse/CALCITE-270")
> -  @Test public void testGroupByHaving2() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, count(*) as c from zips\n"
> -            + "group by state having sum(pop) > 12000000")
> -        .returns("STATE=NY; C=1596\n"
> -            + "STATE=TX; C=1676\n"
> -            + "STATE=FL; C=826\n"
> -            + "STATE=CA; C=1523\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {STATE: '$state', POP: '$pop'}}",
> -                "{$group: {_id: '$STATE', C: {$sum: 1}, _2: {$sum: '$POP'}}}",
> -                "{$project: {STATE: '$_id', C: '$C', _2: '$_2'}}",
> -                "{\n"
> -                    + "  $match: {\n"
> -                    + "    _2: {\n"
> -                    + "      $gt: 12000000\n"
> -                    + "    }\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {STATE: 1, C: 1}}"));
> -  }
> -
> -  @Test public void testGroupByMinMaxSum() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) as c, state,\n"
> -            + " min(pop) as min_pop, max(pop) as max_pop, sum(pop) as sum_pop\n"
> -            + "from zips group by state order by state")
> -        .limit(2)
> -        .returns("C=195; STATE=AK; MIN_POP=0; MAX_POP=32383; SUM_POP=544698\n"
> -            + "C=567; STATE=AL; MIN_POP=0; MAX_POP=44165; SUM_POP=4040587\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {POP: '$pop', STATE: '$state'}}",
> -                "{$group: {_id: '$STATE', C: {$sum: 1}, MIN_POP: {$min: '$POP'}, MAX_POP: {$max: '$POP'}, SUM_POP: {$sum: '$POP'}}}",
> -                "{$project: {STATE: '$_id', C: '$C', MIN_POP: '$MIN_POP', MAX_POP: '$MAX_POP', SUM_POP: '$SUM_POP'}}",
> -                "{$project: {C: 1, STATE: 1, MIN_POP: 1, MAX_POP: 1, SUM_POP: 1}}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Test public void testGroupComposite() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) as c, state, city from zips\n"
> -            + "group by state, city order by c desc limit 2")
> -        .returns("C=93; STATE=TX; CITY=HOUSTON\n"
> -            + "C=56; STATE=CA; CITY=LOS ANGELES\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {CITY: '$city', STATE: '$state'}}",
> -                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}, C: {$sum: 1}}}",
> -                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE', C: '$C'}}",
> -                "{$sort: {C: -1}}",
> -                "{$limit: 2}",
> -                "{$project: {C: 1, STATE: 1, CITY: 1}}"));
> -  }
> -
> -  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> -  @Test public void testDistinctCount() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, count(distinct city) as cdc from zips\n"
> -            + "where state in ('CA', 'TX') group by state order by state")
> -        .returns("STATE=CA; CDC=1072\n"
> -            + "STATE=TX; CDC=1233\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{\n"
> -                    + "  \"$match\": {\n"
> -                    + "    \"$or\": [\n"
> -                    + "      {\n"
> -                    + "        \"state\": \"CA\"\n"
> -                    + "      },\n"
> -                    + "      {\n"
> -                    + "        \"state\": \"TX\"\n"
> -                    + "      }\n"
> -                    + "    ]\n"
> -                    + "  }\n"
> -                    + "}",
> -                "{$project: {CITY: '$city', STATE: '$state'}}",
> -                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
> -                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
> -                "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> -                "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> -                "{$sort: {STATE: 1}}"));
> -  }
> -
> -  @Test public void testDistinctCountOrderBy() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, count(distinct city) as cdc\n"
> -            + "from zips\n"
> -            + "group by state\n"
> -            + "order by cdc desc limit 5")
> -        .returns("STATE=NY; CDC=1370\n"
> -            + "STATE=PA; CDC=1369\n"
> -            + "STATE=TX; CDC=1233\n"
> -            + "STATE=IL; CDC=1148\n"
> -            + "STATE=CA; CDC=1072\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {CITY: '$city', STATE: '$state'}}",
> -                "{$group: {_id: {CITY: '$CITY', STATE: '$STATE'}}}",
> -                "{$project: {_id: 0, CITY: '$_id.CITY', STATE: '$_id.STATE'}}",
> -                "{$group: {_id: '$STATE', CDC: {$sum: {$cond: [ {$eq: ['CITY', null]}, 0, 1]}}}}",
> -                "{$project: {STATE: '$_id', CDC: '$CDC'}}",
> -                "{$sort: {CDC: -1}}",
> -                "{$limit: 5}"));
> -  }
> -
> -  @Ignore("broken; [CALCITE-2115] is logged to fix it")
> -  @Test public void testProject() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, city, 0 as zero from zips order by state, city")
> -        .limit(2)
> -        .returns("STATE=AK; CITY=AKHIOK; ZERO=0\n"
> -            + "STATE=AK; CITY=AKIACHAK; ZERO=0\n")
> -        .queryContains(
> -            mongoChecker(
> -                "{$project: {CITY: '$city', STATE: '$state'}}",
> -                "{$sort: {STATE: 1, CITY: 1}}",
> -                "{$project: {STATE: 1, CITY: 1, ZERO: {$literal: 0}}}"));
> -  }
> -
> -  @Test public void testFilter() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, city from zips where state = 'CA'")
> -        .limit(2)
> -        .returns("STATE=CA; CITY=LOS ANGELES\n"
> -            + "STATE=CA; CITY=LOS ANGELES\n")
> -        .explainContains("PLAN=MongoToEnumerableConverter\n"
> -            + "  MongoProject(STATE=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], CITY=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
> -            + "    MongoFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
> -            + "      MongoTableScan(table=[[mongo_raw, zips]])");
> -  }
> -
> -  /** MongoDB's predicates are handed (they can only accept literals on the
> -   * right-hand size) so it's worth testing that we handle them right both
> -   * ways around. */
> -  @Test public void testFilterReversed() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, city from zips where 'WI' < state")
> -        .limit(2)
> -        .returns("STATE=WV; CITY=BLUEWELL\n"
> -            + "STATE=WV; CITY=ATHENS\n");
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select state, city from zips where state > 'WI'")
> -        .limit(2)
> -        .returns("STATE=WV; CITY=BLUEWELL\n"
> -            + "STATE=WV; CITY=ATHENS\n");
> -  }
> -
> -  /** MongoDB's predicates are handed (they can only accept literals on the
> -   * right-hand size) so it's worth testing that we handle them right both
> -   * ways around.
> -   *
> -   * <p>Test case for
> -   * <a href="https://issues.apache.org/jira/browse/CALCITE-740">[CALCITE-740]
> -   * Redundant WHERE clause causes wrong result in MongoDB adapter</a>. */
> -  @Test public void testFilterPair() {
> -    final int gt9k = 8125;
> -    final int lt9k = 21227;
> -    final int gt8k = 8707;
> -    final int lt8k = 20645;
> -    checkPredicate(gt9k, "where pop > 8000 and pop > 9000");
> -    checkPredicate(gt9k, "where pop > 9000");
> -    checkPredicate(lt9k, "where pop < 9000");
> -    checkPredicate(gt8k, "where pop > 8000");
> -    checkPredicate(lt8k, "where pop < 8000");
> -    checkPredicate(gt9k, "where pop > 9000 and pop > 8000");
> -    checkPredicate(gt8k, "where pop > 9000 or pop > 8000");
> -    checkPredicate(gt8k, "where pop > 8000 or pop > 9000");
> -    checkPredicate(lt8k, "where pop < 8000 and pop < 9000");
> -  }
> -
> -  private void checkPredicate(int expected, String q) {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) as c from zips\n"
> -            + q)
> -        .returns("C=" + expected + "\n");
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select * from zips\n"
> -            + q)
> -        .returnsCount(expected);
> -  }
> -
> -  @Ignore
> -  @Test public void testFoodmartQueries() {
> -    final List<Pair<String, String>> queries = JdbcTest.getFoodmartQueries();
> -    for (Ord<Pair<String, String>> query : Ord.zip(queries)) {
> -//      if (query.i != 29) continue;
> -      if (query.e.left.contains("agg_")) {
> -        continue;
> -      }
> -      final CalciteAssert.AssertQuery query1 =
> -          CalciteAssert.that()
> -              .enable(enabled())
> -              .with(FOODMART)
> -              .query(query.e.left);
> -      if (query.e.right != null) {
> -        query1.returns(query.e.right);
> -      } else {
> -        query1.runs();
> -      }
> -    }
> -  }
> -
> -  /** Test case for
> -   * <a href="https://issues.apache.org/jira/browse/CALCITE-286">[CALCITE-286]
> -   * Error casting MongoDB date</a>. */
> -  @Test public void testDate() {
> -    // Assumes that you have created the following collection before running
> -    // this test:
> -    //
> -    // $ mongo
> -    // > use test
> -    // switched to db test
> -    // > db.createCollection("datatypes")
> -    // { "ok" : 1 }
> -    // > db.datatypes.insert( {
> -    //     "_id" : ObjectId("53655599e4b0c980df0a8c27"),
> -    //     "_class" : "com.ericblue.Test",
> -    //     "date" : ISODate("2012-09-05T07:00:00Z"),
> -    //     "value" : 1231,
> -    //     "ownerId" : "531e7789e4b0853ddb861313"
> -    //   } )
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .withModel("{\n"
> -            + "  version: '1.0',\n"
> -            + "  defaultSchema: 'test',\n"
> -            + "   schemas: [\n"
> -            + "     {\n"
> -            + "       type: 'custom',\n"
> -            + "       name: 'test',\n"
> -            + "       factory: 'org.apache.calcite.adapter.mongodb.MongoSchemaFactory',\n"
> -            + "       operand: {\n"
> -            + "         host: 'localhost',\n"
> -            + "         database: 'test'\n"
> -            + "       }\n"
> -            + "     }\n"
> -            + "   ]\n"
> -            + "}")
> -        .query("select cast(_MAP['date'] as DATE) from \"datatypes\"")
> -        .returnsUnordered("EXPR$0=2012-09-05");
> -  }
> +public class MongoAdapterIT extends MongoAdapterTest {
> 
> -  /** Test case for
> -   * <a href="https://issues.apache.org/jira/browse/CALCITE-665">[CALCITE-665]
> -   * ClassCastException in MongoDB adapter</a>. */
> -  @Test public void testCountViaInt() {
> -    CalciteAssert.that()
> -        .enable(enabled())
> -        .with(ZIPS)
> -        .query("select count(*) from zips")
> -        .returns(
> -            new Function<ResultSet, Void>() {
> -              public Void apply(ResultSet input) {
> -                try {
> -                  assertThat(input.next(), CoreMatchers.is(true));
> -                  assertThat(input.getInt(1), CoreMatchers.is(29353));
> -                  return null;
> -                } catch (SQLException e) {
> -                  throw new RuntimeException(e);
> -                }
> -              }
> -            });
> +  @BeforeClass
> +  public static void enforceMongo() {
> +    assumeTrue(MongoAssertions.useMongo());
>   }
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/calcite/blob/dcf396a5/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
> ----------------------------------------------------------------------
> diff --git a/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java b/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
> new file mode 100644
> index 0000000..028286a
> --- /dev/null
> +++ b/mongodb/src/test/java/org/apache/calcite/test/MongoAssertions.java
> @@ -0,0 +1,101 @@
> +/*
> + * 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.calcite.test;
> +
> +import org.apache.calcite.util.Util;
> +
> +import com.google.common.base.Function;
> +import com.google.common.collect.Lists;
> +import com.google.common.collect.Ordering;
> +
> +import java.sql.ResultSet;
> +import java.sql.SQLException;
> +import java.util.Arrays;
> +import java.util.Collections;
> +import java.util.List;
> +
> +import static org.hamcrest.CoreMatchers.equalTo;
> +import static org.junit.Assert.assertThat;
> +import static org.junit.Assume.assumeTrue;
> +
> +/**
> + * Util class which needs to be in the same package as {@link CalciteAssert}
> + * due to package-private visibility.
> + */
> +public class MongoAssertions {
> +
> +  private MongoAssertions() {}
> +
> +  /** Similar to {@link CalciteAssert#checkResultUnordered}, but filters strings
> +   * before comparing them. */
> +  public static Function<ResultSet, Void> checkResultUnordered(
> +      final String... lines) {
> +    return new Function<ResultSet, Void>() {
> +      public Void apply(ResultSet resultSet) {
> +        try {
> +          final List<String> expectedList =
> +              Ordering.natural().immutableSortedCopy(Arrays.asList(lines));
> +
> +          final List<String> actualList = Lists.newArrayList();
> +          CalciteAssert.toStringList(resultSet, actualList);
> +          for (int i = 0; i < actualList.size(); i++) {
> +            String s = actualList.get(i);
> +            actualList.set(i,
> +                s.replaceAll("\\.0;", ";").replaceAll("\\.0$", ""));
> +          }
> +          Collections.sort(actualList);
> +
> +          assertThat(Ordering.natural().immutableSortedCopy(actualList),
> +              equalTo(expectedList));
> +          return null;
> +        } catch (SQLException e) {
> +          throw new RuntimeException(e);
> +        }
> +      }
> +    };
> +  }
> +
> +  /**
> +   * Whether to run Mongo integration tests. Enabled by default, however test is only
> +   * included if "it" profile is activated ({@code -Pit}). To disable,
> +   * specify {@code -Dcalcite.test.mongodb=false} on the Java command line.
> +   */
> +  public static boolean useMongo() {
> +    return Util.getBooleanProperty("calcite.integrationTest")
> +            && Util.getBooleanProperty("calcite.test.mongodb", true);
> +  }
> +
> +  /**
> +   * Checks wherever tests should use Fongo instead of Mongo. Opposite of {@link #useMongo()}.
> +   */
> +  public static boolean useFongo() {
> +    return !useMongo();
> +  }
> +
> +
> +  /**
> +   * Used to skip tests if current instance is not mongo. Some functionalities
> +   * are not available in fongo.
> +   *
> +   * @see <a href="https://github.com/fakemongo/fongo/issues/152">Aggregation with $cond (172)</a>
> +   */
> +  public static void assumeRealMongoInstance() {
> +    assumeTrue("Expect mongo instance", useMongo());
> +  }
> +}
> +
> +// End MongoAssertions.java
>