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/21 10:37:55 UTC

[2/2] calcite git commit: [CALCITE-2347] running ElasticSearch in embedded mode for unit tests of ES adapter (Andrei Sereda)

[CALCITE-2347] running ElasticSearch in embedded mode for unit tests of ES adapter (Andrei Sereda)

After discussion on dev-list Integration tests (for ES) have been removed. They're now
superseded by unit tests (which execute queries against a real elastic instance)

Added local file (zips-mini.json) which contains a small subset of original zips.json
(allows to bootstrap tests faster)

Created separate ES JUnit rule which can be re-used across different tests.

Both v2 and v5 of ES adapters are supported.


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

Branch: refs/heads/master
Commit: e7b353bb332381ac40b54640c9b13d2bec8b716a
Parents: 37944bb
Author: Andrei Sereda <an...@nospam.com>
Authored: Thu May 31 18:19:10 2018 -0400
Committer: Michael Mior <mm...@uwaterloo.ca>
Committed: Thu Jun 21 06:36:25 2018 -0400

----------------------------------------------------------------------
 .../AbstractElasticsearchTable.java             |  12 +
 .../elasticsearch/ElasticsearchProject.java     |  61 ++-
 elasticsearch2/pom.xml                          |   6 +
 .../Elasticsearch2Enumerator.java               |  12 +-
 .../elasticsearch2/Elasticsearch2Schema.java    |  16 +-
 .../elasticsearch2/Elasticsearch2Table.java     |   9 +-
 .../ElasticSearch2AdapterTest.java              | 395 ++++++++++++++++++
 .../elasticsearch2/EmbeddedElasticNode.java     | 147 +++++++
 .../elasticsearch2/EmbeddedElasticRule.java     |  97 +++++
 .../org/apache/calcite/test/ElasticChecker.java |  49 +++
 .../calcite/test/Elasticsearch2AdapterIT.java   | 270 -------------
 .../resources/elasticsearch-zips-model.json     |  50 ---
 .../src/test/resources/zips-mini.json           | 149 +++++++
 elasticsearch5/pom.xml                          |  31 ++
 .../elasticsearch5/Elasticsearch5Schema.java    |  17 +-
 .../elasticsearch5/Elasticsearch5Table.java     |  11 +-
 .../ElasticSearch5AdapterTest.java              | 399 +++++++++++++++++++
 .../elasticsearch5/EmbeddedElasticNode.java     | 153 +++++++
 .../elasticsearch5/EmbeddedElasticRule.java     |  98 +++++
 .../org/apache/calcite/test/ElasticChecker.java |  49 +++
 .../calcite/test/Elasticsearch5AdapterIT.java   | 270 -------------
 .../resources/elasticsearch-zips-model.json     |  50 ---
 elasticsearch5/src/test/resources/log4j2.xml    |  16 +
 .../src/test/resources/zips-mini.json           | 149 +++++++
 pom.xml                                         |  20 +-
 25 files changed, 1866 insertions(+), 670 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
index 0980469..8cc5933 100644
--- a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/AbstractElasticsearchTable.java
@@ -75,6 +75,18 @@ public abstract class AbstractElasticsearchTable extends AbstractQueryableTable
         relOptTable, this, null);
   }
 
+  /**
+   * In ES 5.x scripted fields start with {@code params._source.foo} while in ES2.x
+   * {@code _source.foo}. Helper method to build correct query based on runtime version of elastic.
+   *
+   * @see <a href="https://github.com/elastic/elasticsearch/issues/20068">_source variable</a>
+   * @see <a href="https://www.elastic.co/guide/en/elasticsearch/reference/master/modules-scripting-fields.html">Scripted Fields</a>
+   */
+  protected String scriptedFieldPrefix() {
+    // this is default pattern starting 5.x
+    return "params._source";
+  }
+
   /** Executes a "find" operation on the underlying type.
    *
    * <p>For example,

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
index b42abd7..961c8b0 100644
--- a/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
+++ b/core/src/main/java/org/apache/calcite/adapter/elasticsearch/ElasticsearchProject.java
@@ -27,11 +27,15 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.List;
 
+import javax.annotation.Nullable;
+
 /**
  * Implementation of {@link org.apache.calcite.rel.core.Project}
  * relational expression in Elasticsearch.
@@ -57,41 +61,60 @@ public class ElasticsearchProject extends Project implements ElasticsearchRel {
     implementor.visitChild(0, getInput());
 
     final List<String> inFields =
-        ElasticsearchRules.elasticsearchFieldNames(getInput().getRowType());
+            ElasticsearchRules.elasticsearchFieldNames(getInput().getRowType());
     final ElasticsearchRules.RexToElasticsearchTranslator translator =
-        new ElasticsearchRules.RexToElasticsearchTranslator(
-            (JavaTypeFactory) getCluster().getTypeFactory(), inFields);
+            new ElasticsearchRules.RexToElasticsearchTranslator(
+                    (JavaTypeFactory) getCluster().getTypeFactory(), inFields);
 
-    final List<String> findItems = new ArrayList<>();
-    final List<String> scriptFieldItems = new ArrayList<>();
+    final List<String> fields = new ArrayList<>();
+    final List<String> scriptFields = new ArrayList<>();
     for (Pair<RexNode, String> pair: getNamedProjects()) {
       final String name = pair.right;
       final String expr = pair.left.accept(translator);
 
       if (expr.equals("\"" + name + "\"")) {
-        findItems.add(ElasticsearchRules.quote(name));
+        fields.add(name);
       } else if (expr.matches("\"literal\":.+")) {
-        scriptFieldItems.add(ElasticsearchRules.quote(name)
-            + ":{\"script\": "
-            + expr.split(":")[1] + "}");
+        scriptFields.add(ElasticsearchRules.quote(name)
+                + ":{\"script\": "
+                + expr.split(":")[1] + "}");
       } else {
-        scriptFieldItems.add(ElasticsearchRules.quote(name)
-            + ":{\"script\":\"params._source."
-            + expr.replaceAll("\"", "") + "\"}");
+        scriptFields.add(ElasticsearchRules.quote(name)
+                + ":{\"script\":"
+                // _source (ES2) vs params._source (ES5)
+                + "\"" + implementor.elasticsearchTable.scriptedFieldPrefix() + "."
+                + expr.replaceAll("\"", "") + "\"}");
+      }
+    }
+
+    StringBuilder query = new StringBuilder();
+    if (scriptFields.isEmpty()) {
+      List<String> newList = Lists.transform(fields, new Function<String, String>() {
+        @Nullable
+        @Override public String apply(@Nullable String input) {
+          return ElasticsearchRules.quote(input);
+        }
+      });
+
+      final String findString = String.join(", ", newList);
+      query.append("\"_source\" : [").append(findString).append("]");
+    } else {
+      // if scripted fields are present, ES ignores _source attribute
+      for (String field: fields) {
+        scriptFields.add(ElasticsearchRules.quote(field) + ":{\"script\": "
+                // _source (ES2) vs params._source (ES5)
+                + "\"" + implementor.elasticsearchTable.scriptedFieldPrefix() + "."
+                + field + "\"}");
       }
+      query.append("\"script_fields\": {" + String.join(", ", scriptFields) + "}");
     }
-    final String findString = Util.toString(findItems, "", ", ", "");
-    final String scriptFieldString = "\"script_fields\": {"
-        + Util.toString(scriptFieldItems, "", ", ", "") + "}";
-    final String fieldString = "\"_source\" : [" + findString + "]"
-        + ", " + scriptFieldString;
 
     for (String opfield : implementor.list) {
       if (opfield.startsWith("\"_source\"")) {
         implementor.list.remove(opfield);
       }
     }
-    implementor.add(fieldString);
+    implementor.add(query.toString());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/pom.xml
----------------------------------------------------------------------
diff --git a/elasticsearch2/pom.xml b/elasticsearch2/pom.xml
index f24622c..6fbee03 100644
--- a/elasticsearch2/pom.xml
+++ b/elasticsearch2/pom.xml
@@ -73,6 +73,12 @@ limitations under the License.
       <version>${elasticsearch-java-driver.version}</version>
     </dependency>
     <dependency>
+      <!-- Lang groovy dependency is needed for testing with embedded ES (scripted fields like loc[0]) -->
+      <groupId>org.elasticsearch.module</groupId>
+      <artifactId>lang-groovy</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>com.carrotsearch</groupId>
       <artifactId>hppc</artifactId>
       <version>${hppc.version}</version>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
index 84370ab..c3d2ac0 100644
--- a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Enumerator.java
@@ -26,6 +26,7 @@ import org.elasticsearch.search.SearchHit;
 import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 /**
@@ -101,15 +102,18 @@ public class Elasticsearch2Enumerator implements Enumerator<Object> {
   private static Function1<SearchHit, Object[]> listGetter(
       final List<Map.Entry<String, Class>> fields) {
     return new Function1<SearchHit, Object[]>() {
-      public Object[] apply(SearchHit searchHitFields) {
+      public Object[] apply(SearchHit hit) {
         Object[] objects = new Object[fields.size()];
         for (int i = 0; i < fields.size(); i++) {
           final Map.Entry<String, Class> field = fields.get(i);
           final String name = field.getKey();
-          if (searchHitFields.fields().isEmpty()) {
-            objects[i] = convert(searchHitFields.getSource().get(name), field.getValue());
+          if (hit.fields().isEmpty()) {
+            objects[i] = convert(hit.getSource().get(name), field.getValue());
+          } else if (hit.fields().containsKey(name)) {
+            objects[i] = convert(hit.field(name).getValue(), field.getValue());
           } else {
-            objects[i] = convert(searchHitFields.field(name).getValue(), field.getValue());
+            throw new IllegalStateException(
+                    String.format(Locale.getDefault(), "No result for %s", field));
           }
         }
         return objects;

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
index 668402b..46e3fc5 100644
--- a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Schema.java
@@ -22,6 +22,9 @@ import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
@@ -86,6 +89,16 @@ public class Elasticsearch2Schema extends AbstractSchema
     }
   }
 
+  /**
+   * Allows schema to be instantiated from existing elastic search client.
+   * This constructor is used in tests.
+   */
+  @VisibleForTesting
+  Elasticsearch2Schema(Client client, String index) {
+    this.client = Preconditions.checkNotNull(client, "client");
+    this.index = Preconditions.checkNotNull(index, "index");
+  }
+
   @Override protected Map<String, Table> getTableMap() {
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
 
@@ -120,7 +133,8 @@ public class Elasticsearch2Schema extends AbstractSchema
 
     final List<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes());
     if (nodes.isEmpty()) {
-      throw new RuntimeException("Cannot connect to any elasticsearch nodes");
+      throw new IllegalStateException("Cannot connect to any elasticsearch node: "
+              + transportNodes);
     }
 
     client = transportClient;

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
index 636aa5f..2928835 100644
--- a/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
+++ b/elasticsearch2/src/main/java/org/apache/calcite/adapter/elasticsearch2/Elasticsearch2Table.java
@@ -45,8 +45,15 @@ public class Elasticsearch2Table extends AbstractElasticsearchTable {
     this.client = client;
   }
 
+  /**
+   * ES version 2.x. To access document attributes ES2 uses {@code _source.foo} syntax.
+   */
+  @Override protected String scriptedFieldPrefix() {
+    return "_source";
+  }
+
   @Override protected Enumerable<Object> find(String index, List<String> ops,
-      List<Map.Entry<String, Class>> fields) {
+                                              List<Map.Entry<String, Class>> fields) {
     final String dbName = index;
 
     final String queryString = "{" + Util.toString(ops, "", ", ", "") + "}";

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/ElasticSearch2AdapterTest.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/ElasticSearch2AdapterTest.java b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/ElasticSearch2AdapterTest.java
new file mode 100644
index 0000000..287e094
--- /dev/null
+++ b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/ElasticSearch2AdapterTest.java
@@ -0,0 +1,395 @@
+/*
+ * 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.elasticsearch2;
+
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.calcite.schema.impl.ViewTableMacro;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.ElasticChecker;
+
+import com.google.common.io.LineProcessor;
+import com.google.common.io.Resources;
+
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Locale;
+
+/**
+ * Set of tests for ES adapter. Uses real instance via {@link EmbeddedElasticRule}. Document
+ * source is local {@code zips-mini.json} file (located in the classpath).
+ */
+public class ElasticSearch2AdapterTest {
+
+  @ClassRule //init once for all tests
+  public static final EmbeddedElasticRule NODE = EmbeddedElasticRule.create();
+
+  private static final String ZIPS = "zips";
+
+  /**
+   * Used to create {@code zips} index and insert some data
+   */
+  @BeforeClass
+  public static void setupInstance() throws Exception {
+    // define mapping so fields are searchable (term query)
+    XContentBuilder mapping = XContentFactory.jsonBuilder().startObject()
+            .startObject("properties")
+            .startObject("city").field("type", "string")
+                  .field("index", "not_analyzed").endObject()
+            .startObject("state").field("type", "string")
+                  .field("index", "not_analyzed").endObject()
+            .startObject("pop").field("type", "long").endObject()
+            .endObject()
+            .endObject();
+
+    // create index
+    NODE.client().admin().indices()
+            .prepareCreate(ZIPS)
+            .addMapping(ZIPS, mapping)
+            .get();
+
+    BulkRequestBuilder bulk = NODE.client().prepareBulk().setRefresh(true);
+
+    // load records from file
+    Resources.readLines(ElasticSearch2AdapterTest.class.getResource("/zips-mini.json"),
+            StandardCharsets.UTF_8, new LineProcessor<Void>() {
+              @Override public boolean processLine(String line) throws IOException {
+                line = line.replaceAll("_id", "id"); // _id is a reserved attribute in ES
+                bulk.add(NODE.client().prepareIndex(ZIPS, ZIPS).setSource(line));
+                return true;
+              }
+
+              @Override public Void getResult() {
+                return null;
+              }
+            });
+
+    if (bulk.numberOfActions() == 0) {
+      throw new IllegalStateException("No records to be indexed");
+    }
+
+    BulkResponse response = bulk.execute().get();
+
+    if (response.hasFailures()) {
+      throw new IllegalStateException(
+              String.format(Locale.getDefault(), "Failed to populate %s:\n%s", NODE.httpAddress(),
+              Arrays.stream(response.getItems()).filter(BulkItemResponse::isFailed)
+                      .map(BulkItemResponse::getFailureMessage).findFirst().orElse("<unknown>")));
+    }
+
+  }
+
+  private CalciteAssert.ConnectionFactory newConnectionFactory() {
+    return new CalciteAssert.ConnectionFactory() {
+      @Override public Connection createConnection() throws SQLException {
+        final Connection connection = DriverManager.getConnection("jdbc:calcite:");
+        final SchemaPlus root = connection.unwrap(CalciteConnection.class).getRootSchema();
+
+        root.add("elastic", new Elasticsearch2Schema(NODE.client(), ZIPS));
+
+        // add calcite view programmatically
+        final String viewSql = "select cast(_MAP['city'] AS varchar(20)) AS \"city\", "
+               + " cast(_MAP['loc'][0] AS float) AS \"longitude\",\n"
+               + " cast(_MAP['loc'][1] AS float) AS \"latitude\",\n"
+              + " cast(_MAP['pop'] AS integer) AS \"pop\", "
+              +  " cast(_MAP['state'] AS varchar(2)) AS \"state\", "
+              +  " cast(_MAP['id'] AS varchar(5)) AS \"id\" "
+              +  "from \"elastic\".\"zips\"";
+
+        ViewTableMacro macro = ViewTable.viewMacro(root, viewSql,
+                Collections.singletonList("elastic"), Arrays.asList("elastic", "view"), false);
+        root.add("ZIPS", macro);
+
+        return connection;
+      }
+    };
+  }
+
+  private CalciteAssert.AssertThat calciteAssert() {
+    return CalciteAssert.that()
+            .with(newConnectionFactory());
+  }
+
+  /**
+   * Tests using calcite view
+   */
+  @Test
+  public void view() throws Exception {
+    calciteAssert()
+          .query("select * from zips where \"city\" = 'BROOKLYN'")
+          .returns("city=BROOKLYN; longitude=-73.956985; latitude=40.646694; "
+                  + "pop=111396; state=NY; id=11226\n")
+          .returnsCount(1);
+  }
+
+  @Test
+  public void emptyResult() {
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from zips limit 0")
+            .returnsCount(0);
+
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" where _MAP['Foo'] = '_MISSING_'")
+            .returnsCount(0);
+  }
+
+  @Test
+  public void basic() throws Exception {
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" where _MAP['city'] = 'BROOKLYN'")
+            .returnsCount(1);
+
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" where"
+                    + " _MAP['city'] in ('BROOKLYN', 'WASHINGTON')")
+            .returnsCount(2);
+
+    // lower-case
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" where "
+                    + "_MAP['city'] in ('brooklyn', 'Brooklyn', 'BROOK') ")
+            .returnsCount(0);
+
+    // missing field
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" where _MAP['CITY'] = 'BROOKLYN'")
+            .returnsCount(0);
+
+    // limit works
+    CalciteAssert.that()
+            .with(newConnectionFactory())
+            .query("select * from \"elastic\".\"zips\" limit 42")
+            .returnsCount(42);
+
+  }
+
+  @Test public void testSort() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+            + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
+            + "    ElasticsearchProject(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"
+            + "      ElasticsearchTableScan(table=[[elastic, zips]])";
+
+    calciteAssert()
+            .query("select * from zips order by \"state\"")
+            .returnsCount(10)
+            .explainContains(explain);
+  }
+
+  @Test public void testSortLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+            + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
+    calciteAssert()
+            .query(sql)
+            .returnsUnordered("state=AK; id=99801",
+                    "state=AL; id=35215",
+                    "state=AL; id=35401")
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker(
+                            "\"_source\" : [\"state\", \"id\"]",
+                            "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
+                            "\"from\": 2",
+                            "\"size\": 3"));
+  }
+
+
+
+  @Test public void testOffsetLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+            + "offset 2 fetch next 3 rows only";
+    calciteAssert()
+            .query(sql)
+            .runs()
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker(
+                            "\"from\": 2",
+                            "\"size\": 3",
+                            "\"_source\" : [\"state\", \"id\"]"));
+  }
+
+  @Test public void testLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+            + "fetch next 3 rows only";
+
+    calciteAssert()
+            .query(sql)
+            .runs()
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker(
+                            "\"size\": 3",
+                            "\"_source\" : [\"state\", \"id\"]"));
+  }
+
+  @Test public void testFilterSort() {
+    final String sql = "select * from zips\n"
+            + "where \"state\" = 'CA' and \"id\" >= '70000'\n"
+            + "order by \"state\", \"id\"";
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+            + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
+            + "    ElasticsearchProject(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"
+            + "      ElasticsearchFilter(condition=[AND(=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA'), >=(CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
+            + "        ElasticsearchTableScan(table=[[elastic, zips]])";
+    calciteAssert()
+            .query(sql)
+            .returnsOrdered("city=LOS ANGELES; longitude=-118.258189; latitude=34.007856; "
+                            + "pop=96074; state=CA; id=90011",
+                    "city=BELL GARDENS; longitude=-118.17205; latitude=33.969177; "
+                            + "pop=99568; state=CA; id=90201",
+                    "city=NORWALK; longitude=-118.081767; latitude=33.90564; "
+                            + "pop=94188; state=CA; id=90650")
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker("\"query\" : "
+                                    + "{\"constant_score\":{\"filter\":{\"bool\":"
+                                    + "{\"must\":[{\"term\":{\"state\":\"CA\"}},"
+                                    + "{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
+                            "\"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, "
+                                    + "\"latitude\":{\"script\":\"_source.loc[1]\"}, "
+                                    + "\"city\":{\"script\": \"_source.city\"}, "
+                                    + "\"pop\":{\"script\": \"_source.pop\"}, "
+                                    + "\"state\":{\"script\": \"_source.state\"}, "
+                                    + "\"id\":{\"script\": \"_source.id\"}}",
+                            "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
+            .explainContains(explain);
+  }
+
+  @Test public void testFilterSortDesc() {
+    final String sql = "select * from zips\n"
+            + "where \"pop\" BETWEEN 95000 AND 100000\n"
+            + "order by \"state\" desc, \"pop\"";
+    calciteAssert()
+            .query(sql)
+            .limit(4)
+            .returnsOrdered(
+             "city=LOS ANGELES; longitude=-118.258189; latitude=34.007856; pop=96074; state=CA; id=90011",
+             "city=BELL GARDENS; longitude=-118.17205; latitude=33.969177; pop=99568; state=CA; id=90201");
+  }
+
+  @Test public void testFilterRedundant() {
+    final String sql = "select * from zips\n"
+            + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
+    calciteAssert()
+            .query(sql)
+            .runs()
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker(""
+                                    + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                                    + "{\"must\":[{\"term\":{\"state\":\"OK\"}}]}}}}",
+                            "\"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, "
+                                    +  "\"latitude\":{\"script\":\"_source.loc[1]\"}, "
+                                    +   "\"city\":{\"script\": \"_source.city\"}, "
+                                    +   "\"pop\":{\"script\": \"_source.pop\"}, \"state\":{\"script\": \"_source.state\"}, "
+                                    +            "\"id\":{\"script\": \"_source.id\"}}"
+                    ));
+  }
+
+  @Test public void testInPlan() {
+    final String[] searches = {
+        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
+                 + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":96074}}]}},{\"bool\":{\"must\":[{\"term\":"
+                 + "{\"pop\":99568}}]}}]}}}}",
+        "\"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, "
+                 +  "\"latitude\":{\"script\":\"_source.loc[1]\"}, "
+                 +  "\"city\":{\"script\": \"_source.city\"}, "
+                 +  "\"pop\":{\"script\": \"_source.pop\"}, "
+                 +  "\"state\":{\"script\": \"_source.state\"}, "
+                 +  "\"id\":{\"script\": \"_source.id\"}}"
+    };
+
+    calciteAssert()
+            .query("select * from zips where \"pop\" in (96074, 99568)")
+            .returnsUnordered(
+                    "city=BELL GARDENS; longitude=-118.17205; latitude=33.969177; pop=99568; state=CA; id=90201",
+                    "city=LOS ANGELES; longitude=-118.258189; latitude=34.007856; pop=96074; state=CA; id=90011"
+                    )
+            .queryContains(ElasticChecker.elasticsearchChecker(searches));
+  }
+
+  @Test public void testZips() {
+    calciteAssert()
+         .query("select \"state\", \"city\" from zips")
+         .returnsCount(10);
+  }
+
+  @Test public void testProject() {
+    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
+            + "from zips\n"
+            + "order by \"state\", \"city\"";
+
+    calciteAssert()
+            .query(sql)
+            .limit(2)
+            .returnsUnordered("state=AK; city=ANCHORAGE; zero=0",
+                    "state=AK; city=FAIRBANKS; zero=0")
+            .queryContains(
+                    ElasticChecker.elasticsearchChecker("\"script_fields\": "
+                                    + "{\"zero\":{\"script\": \"0\"}, "
+                                    + "\"state\":{\"script\": \"_source.state\"}, "
+                                    + "\"city\":{\"script\": \"_source.city\"}}",
+                            "\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]"));
+  }
+
+  @Test public void testFilter() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+            + "  ElasticsearchProject(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"
+            + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+            + "      ElasticsearchTableScan(table=[[elastic, zips]])";
+    calciteAssert()
+            .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
+            .limit(3)
+            .returnsUnordered("state=CA; city=BELL GARDENS",
+                    "state=CA; city=LOS ANGELES",
+                    "state=CA; city=NORWALK")
+            .explainContains(explain);
+  }
+
+  @Test public void testFilterReversed() {
+    calciteAssert()
+          .query("select \"state\", \"city\" from zips where 'WI' < \"state\" order by \"city\"")
+          .limit(2)
+          .returnsUnordered("state=WV; city=BECKLEY",
+                    "state=WY; city=CHEYENNE");
+    calciteAssert()
+          .query("select \"state\", \"city\" from zips where \"state\" > 'WI' order by \"city\"")
+          .limit(2)
+          .returnsUnordered("state=WV; city=BECKLEY",
+                    "state=WY; city=CHEYENNE");
+  }
+
+}
+
+// End ElasticSearch2AdapterTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticNode.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticNode.java b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticNode.java
new file mode 100644
index 0000000..4474add
--- /dev/null
+++ b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticNode.java
@@ -0,0 +1,147 @@
+/*
+ * 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.elasticsearch2;
+
+import com.google.common.base.Preconditions;
+import com.google.common.io.Files;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
+import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.internal.InternalSettingsPreparer;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.script.groovy.GroovyPlugin;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Represents a single elastic search node which can run embedded in a java application.
+ * Intended for unit and integration tests. Settings and plugins are crafted for Calcite.
+ */
+class EmbeddedElasticNode implements AutoCloseable {
+
+  private final LocalNode node;
+  private volatile boolean  isStarted;
+
+  private EmbeddedElasticNode(LocalNode node) {
+    this.node = Preconditions.checkNotNull(node, "node");
+  }
+
+  /**
+   * Having separate class to expose (protected) constructor which allows to install
+   * different plugins. In our case it is {@code GroovyPlugin} for scripted fields like
+   * {@code loc[0]} or {@code loc[1]['foo']}.
+   */
+  private static class LocalNode extends Node {
+    private LocalNode(Settings settings, Collection<Class<? extends Plugin>> classpathPlugins) {
+      super(InternalSettingsPreparer.prepareEnvironment(settings, null),
+              Version.CURRENT,
+              classpathPlugins);
+    }
+  }
+
+  /**
+   * Creates an instance with existing settings
+   */
+  private static EmbeddedElasticNode create(Settings settings) {
+    // ensure GroovyPlugin is installed or otherwise scripted fields would not work
+    LocalNode node = new LocalNode(settings, Collections.singleton(GroovyPlugin.class));
+    return new EmbeddedElasticNode(node);
+  }
+
+  /**
+   * Creates elastic node as single member of a cluster. Node will not be started
+   * unless {@link #start()} is explicitly called.
+   */
+  public static EmbeddedElasticNode create() {
+    File data = Files.createTempDir();
+    data.deleteOnExit();
+    File home = Files.createTempDir();
+    home.deleteOnExit();
+
+    Settings settings = Settings.builder()
+            .put("node.name", "fake-elastic")
+            .put("path.home", home.getAbsolutePath())
+            .put("path.data", data.getAbsolutePath())
+            .put("script.inline", true)  // requires GroovyPlugin
+            .put("script.indexed", true) // requires GroovyPlugin
+            .put("cluster.routing.allocation.disk.threshold_enabled", false)
+            .put("node.local", true)
+            .put("node.data", true)
+            .put("network.host", "localhost")
+            .build();
+
+    return create(settings);
+  }
+
+  /**
+   * Starts current node
+   */
+  public void start() {
+    Preconditions.checkState(!isStarted, "already started");
+    node.start();
+    this.isStarted = true;
+  }
+
+  /**
+   * Returns current address to connect to with HTTP client.
+   */
+  public TransportAddress httpAddress() {
+    Preconditions.checkState(isStarted, "node is not started");
+
+    NodesInfoResponse response =  client().admin().cluster().prepareNodesInfo()
+            .execute().actionGet();
+    if (response.getNodes().length != 1) {
+      throw new IllegalStateException("Expected single node but got "
+              + response.getNodes().length);
+    }
+    NodeInfo node = response.getNodes()[0];
+    return node.getHttp().address().boundAddresses()[0];
+  }
+
+  /**
+   * Exposes elastic
+   * <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/transport-client.html">transport client</a>
+   *
+   * (use of HTTP client is preferred).
+   */
+  public Client client() {
+    Preconditions.checkState(isStarted, "node is not started");
+    return node.client();
+  }
+
+  @Override public void close() throws Exception {
+    node.close();
+    // cleanup data dirs
+    File data = new File(node.settings().get("path.data"));
+    File home = new File(node.settings().get("path.home"));
+    for (File file: Arrays.asList(data, home)) {
+      if (file.exists()) {
+        file.delete();
+      }
+    }
+  }
+}
+
+// End EmbeddedElasticNode.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticRule.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticRule.java b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticRule.java
new file mode 100644
index 0000000..a633078
--- /dev/null
+++ b/elasticsearch2/src/test/java/org/apache/calcite/adapter/elasticsearch2/EmbeddedElasticRule.java
@@ -0,0 +1,97 @@
+/*
+ * 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.elasticsearch2;
+
+import com.google.common.base.Preconditions;
+
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.junit.rules.ExternalResource;
+
+/**
+ * Used to initialize a single elastic node. For performance reasons (node startup costs),
+ * same instance is usually shared across multiple tests.
+ *
+ * This rule should be used as follows:
+ * <pre>
+ *  {@code
+ *
+ *  public class MyTest {
+ *    @literal @ClassRule
+ *    public static final ElasticSearchRule RULE = ElasticSearchRule.create();
+ *
+ *    @literal @BeforeClass
+ *    public void setup() {
+ *       // ... populate instance
+ *    }
+ *
+ *    @literal @Test
+ *    public void myTest() {
+ *      TransportAddress address = RULE.httpAddress();
+ *      // ....
+ *    }
+ *  }
+ *  }
+ * </pre>
+ *
+ * @see ExternalResource
+ */
+class EmbeddedElasticRule extends ExternalResource {
+
+  private final EmbeddedElasticNode node;
+
+  private EmbeddedElasticRule(EmbeddedElasticNode resource) {
+    this.node = Preconditions.checkNotNull(resource, "resource");
+  }
+
+  @Override protected void before() throws Throwable {
+    node.start();
+  }
+
+  @Override protected void after() {
+    try {
+      node.close();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Factory method to create this rule.
+   */
+  public static EmbeddedElasticRule create() {
+    return new EmbeddedElasticRule(EmbeddedElasticNode.create());
+  }
+
+  /**
+   * Exposes current ES transport client.
+   */
+  Client client() {
+    return node.client();
+  }
+
+  /**
+   * HTTP address for rest clients (can be ES native or any other).
+   */
+  TransportAddress httpAddress() {
+    return node.httpAddress();
+  }
+
+
+}
+
+// End EmbeddedElasticRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/java/org/apache/calcite/test/ElasticChecker.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/test/ElasticChecker.java b/elasticsearch2/src/test/java/org/apache/calcite/test/ElasticChecker.java
new file mode 100644
index 0000000..21fc491
--- /dev/null
+++ b/elasticsearch2/src/test/java/org/apache/calcite/test/ElasticChecker.java
@@ -0,0 +1,49 @@
+/*
+ * 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 com.google.common.base.Function;
+
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+/**
+ * Internal util methods for ElasticSearch tests
+ */
+public class ElasticChecker {
+
+  private ElasticChecker() {}
+
+
+  /** Returns a function that checks that a particular Elasticsearch pipeline is
+   * generated to implement a query. */
+  public static Function<List, Void> elasticsearchChecker(final String... strings) {
+    return new Function<List, Void>() {
+      @Nullable
+      @Override public Void apply(@Nullable List actual) {
+        Object[] actualArray = actual == null || actual.isEmpty() ? null
+            : ((List) actual.get(0)).toArray();
+        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
+            actualArray);
+        return null;
+      }
+    };
+  }
+}
+
+// End ElasticChecker.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java b/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
deleted file mode 100644
index 4e0c2b6..0000000
--- a/elasticsearch2/src/test/java/org/apache/calcite/test/Elasticsearch2AdapterIT.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.test;
-
-import org.apache.calcite.util.Util;
-
-import com.google.common.base.Function;
-import com.google.common.collect.ImmutableMap;
-
-import org.junit.Test;
-
-import java.util.List;
-import javax.annotation.Nullable;
-
-/**
- * Tests for the {@code org.apache.calcite.adapter.elasticsearch2} package.
- *
- * <p>Before calling this test, you need to populate Elasticsearch, as follows:
- *
- * <blockquote><code>
- * git clone https://github.com/vlsi/calcite-test-dataset<br>
- * cd calcite-test-dataset<br>
- * mvn install
- * </code></blockquote>
- *
- * <p>This will create a virtual machine with Elasticsearch and the "zips" test
- * dataset.
- */
-public class Elasticsearch2AdapterIT {
-  /**
-   * Whether to run Elasticsearch tests. Enabled by default, however test is only
-   * included if "it" profile is activated ({@code -Pit}). To disable,
-   * specify {@code -Dcalcite.test.elasticsearch=false} on the Java command line.
-   */
-  private static final boolean ENABLED = Util.getBooleanProperty("calcite.test.elasticsearch",
-      true);
-
-  /** Connection factory based on the "zips-es" model. */
-  private static final ImmutableMap<String, String> ZIPS = ImmutableMap.of("model",
-      Elasticsearch2AdapterIT.class.getResource("/elasticsearch-zips-model.json").getPath());
-
-  /** Whether to run this test. */
-  private boolean enabled() {
-    return ENABLED;
-  }
-
-  /** Returns a function that checks that a particular Elasticsearch pipeline is
-   * generated to implement a query. */
-  private static Function<List, Void> elasticsearchChecker(final String... strings) {
-    return new Function<List, Void>() {
-      @Nullable
-      @Override public Void apply(@Nullable List actual) {
-        Object[] actualArray = actual == null || actual.isEmpty() ? null
-            : ((List) actual.get(0)).toArray();
-        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
-            actualArray);
-        return null;
-      }
-    };
-  }
-
-  @Test public void testSort() {
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
-        + "    ElasticsearchProject(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"
-        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips order by \"state\"")
-        .returnsCount(10)
-        .explainContains(explain);
-  }
-
-  @Test public void testSortLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
-    CalciteAssert.that()
-        .with(ZIPS)
-        .query(sql)
-        .returnsUnordered("state=AK; id=99503",
-            "state=AK; id=99504",
-            "state=AK; id=99505")
-        .queryContains(
-            elasticsearchChecker(
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}",
-                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
-                "\"from\": 2",
-                "\"size\": 3"));
-  }
-
-  @Test public void testOffsetLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "offset 2 fetch next 3 rows only";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(
-                "\"from\": 2",
-                "\"size\": 3",
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
-  }
-
-  @Test public void testLimit() {
-    final String sql = "select \"state\", \"id\" from zips\n"
-        + "fetch next 3 rows only";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(
-                "\"size\": 3",
-                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
-  }
-
-  @Test public void testFilterSort() {
-    final String sql = "select * from zips\n"
-        + "where \"city\" = 'SPRINGFIELD' and \"id\" >= '70000'\n"
-        + "order by \"state\", \"id\"";
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
-        + "    ElasticsearchProject(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"
-        + "      ElasticsearchFilter(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"
-        + "        ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .returnsOrdered(
-            "city=SPRINGFIELD; longitude=-92.54567; latitude=35.274879; pop=752; state=AR; id=72157",
-            "city=SPRINGFIELD; longitude=-102.617322; latitude=37.406727; pop=1992; state=CO; id=81073",
-            "city=SPRINGFIELD; longitude=-90.577479; latitude=30.415738; pop=5597; state=LA; id=70462",
-            "city=SPRINGFIELD; longitude=-123.015259; latitude=44.06106; pop=32384; state=OR; id=97477",
-            "city=SPRINGFIELD; longitude=-122.917108; latitude=44.056056; pop=27521; state=OR; id=97478")
-        .queryContains(
-            elasticsearchChecker("\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
-                    + "{\"must\":[{\"term\":{\"city\":\"springfield\"}},{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
-                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}",
-                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
-        .explainContains(explain);
-  }
-
-  @Test public void testFilterSortDesc() {
-    final String sql = "select * from zips\n"
-        + "where \"pop\" BETWEEN 20000 AND 20100\n"
-        + "order by \"state\" desc, \"pop\"";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .limit(4)
-        .returnsOrdered(
-            "city=SHERIDAN; longitude=-106.964795; latitude=44.78486; pop=20025; state=WY; id=82801",
-            "city=MOUNTLAKE TERRAC; longitude=-122.304036; latitude=47.793061; pop=20059; state=WA; id=98043",
-            "city=FALMOUTH; longitude=-77.404537; latitude=38.314557; pop=20039; state=VA; id=22405",
-            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104");
-  }
-
-  @Test public void testFilterRedundant() {
-    final String sql = "select * from zips\n"
-        + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .runs()
-        .queryContains(
-            elasticsearchChecker(""
-                + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
-                + "{\"must\":[{\"term\":{\"state\":\"ok\"}}]}}}}",
-                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"));
-  }
-
-  @Test public void testInPlan() {
-    final String[] searches = {
-        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
-          + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":20012}}]}},{\"bool\":{\"must\":[{\"term\":"
-          + "{\"pop\":15590}}]}}]}}}}",
-        "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"
-    };
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select * from zips where \"pop\" in (20012, 15590)")
-        .returnsUnordered(
-            "city=COVINA; longitude=-117.884285; latitude=34.08596; pop=15590; state=CA; id=91723",
-            "city=ARLINGTON; longitude=-97.091987; latitude=32.654752; pop=15590; state=TX; id=76018",
-            "city=CROFTON; longitude=-76.680166; latitude=39.011163; pop=15590; state=MD; id=21114",
-            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104",
-            "city=DINUBA; longitude=-119.39087; latitude=36.534931; pop=20012; state=CA; id=93618")
-        .queryContains(elasticsearchChecker(searches));
-  }
-
-  @Test public void testZips() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips")
-        .returnsCount(10);
-  }
-
-  @Test public void testProject() {
-    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
-        + "from zips\n"
-        + "order by \"state\", \"city\"";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query(sql)
-        .limit(2)
-        .returnsUnordered("state=AK; city=ELMENDORF AFB; zero=0",
-            "state=AK; city=EIELSON AFB; zero=0")
-        .queryContains(
-            elasticsearchChecker("\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]",
-                "\"fields\" : [\"state\", \"city\"], \"script_fields\": {\"zero\":{\"script\": \"0\"}}"));
-  }
-
-  @Test public void testFilter() {
-    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
-        + "  ElasticsearchProject(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"
-        + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
-        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
-        .limit(2)
-        .returnsUnordered("state=CA; city=LOS ANGELES",
-            "state=CA; city=LOS ANGELES")
-        .explainContains(explain);
-  }
-
-  @Test public void testFilterReversed() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where 'WI' < \"state\"")
-        .limit(2)
-        .returnsUnordered("state=WV; city=WELCH",
-            "state=WV; city=HANOVER");
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(ZIPS)
-        .query("select \"state\", \"city\" from zips where \"state\" > 'WI'")
-        .limit(2)
-        .returnsUnordered("state=WV; city=WELCH",
-            "state=WV; city=HANOVER");
-  }
-}
-
-// End Elasticsearch2AdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/resources/elasticsearch-zips-model.json b/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
deleted file mode 100644
index 6c00a9c..0000000
--- a/elasticsearch2/src/test/resources/elasticsearch-zips-model.json
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the License); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an AS IS BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-{
-  "version": "1.0",
-  "defaultSchema": "elasticsearch",
-  "schemas": [
-    {
-      "type": "custom",
-      "name": "elasticsearch_raw",
-      "factory": "org.apache.calcite.adapter.elasticsearch2.Elasticsearch2SchemaFactory",
-      "operand": {
-        "coordinates": "{'127.0.0.1': 9300}",
-        "userConfig": "{'bulk.flush.max.actions': 10, 'bulk.flush.max.size.mb': 1}",
-        "index": "usa"
-      }
-    },
-    {
-      "name": "elasticsearch",
-      "tables": [
-        {
-          "name": "ZIPS",
-          "type": "view",
-          "sql": [
-            "select cast(_MAP['city'] AS varchar(20)) AS \"city\",\n",
-            " cast(_MAP['loc'][0] AS float) AS \"longitude\",\n",
-            " cast(_MAP['loc'][1] AS float) AS \"latitude\",\n",
-            " cast(_MAP['pop'] AS integer) AS \"pop\",\n",
-            " cast(_MAP['state'] AS varchar(2)) AS \"state\",\n",
-            " cast(_MAP['id'] AS varchar(5)) AS \"id\"\n",
-            "from \"elasticsearch_raw\".\"zips\""
-          ]
-        }
-      ]
-    }
-  ]
-}

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch2/src/test/resources/zips-mini.json
----------------------------------------------------------------------
diff --git a/elasticsearch2/src/test/resources/zips-mini.json b/elasticsearch2/src/test/resources/zips-mini.json
new file mode 100644
index 0000000..d70eadc
--- /dev/null
+++ b/elasticsearch2/src/test/resources/zips-mini.json
@@ -0,0 +1,149 @@
+{ "_id" : "01701", "city" : "FRAMINGHAM", "loc" : [ -71.42548600000001, 42.300665 ], "pop" : 65046, "state" : "MA" }
+{ "_id" : "02154", "city" : "NORTH WALTHAM", "loc" : [ -71.236497, 42.382492 ], "pop" : 57871, "state" : "MA" }
+{ "_id" : "02401", "city" : "BROCKTON", "loc" : [ -71.03434799999999, 42.081571 ], "pop" : 59498, "state" : "MA" }
+{ "_id" : "02840", "city" : "MIDDLETOWN", "loc" : [ -71.30347999999999, 41.504502 ], "pop" : 47687, "state" : "RI" }
+{ "_id" : "02860", "city" : "PAWTUCKET", "loc" : [ -71.39071300000001, 41.872873 ], "pop" : 45442, "state" : "RI" }
+{ "_id" : "02895", "city" : "NORTH SMITHFIELD", "loc" : [ -71.513683, 41.99948 ], "pop" : 53733, "state" : "RI" }
+{ "_id" : "03060", "city" : "NASHUA", "loc" : [ -71.466684, 42.756395 ], "pop" : 41438, "state" : "NH" }
+{ "_id" : "03103", "city" : "MANCHESTER", "loc" : [ -71.449325, 42.965563 ], "pop" : 36613, "state" : "NH" }
+{ "_id" : "03301", "city" : "CONCORD", "loc" : [ -71.527734, 43.218525 ], "pop" : 34035, "state" : "NH" }
+{ "_id" : "04240", "city" : "LEWISTON", "loc" : [ -70.191619, 44.098538 ], "pop" : 40173, "state" : "ME" }
+{ "_id" : "04401", "city" : "BANGOR", "loc" : [ -68.791839, 44.824199 ], "pop" : 40434, "state" : "ME" }
+{ "_id" : "05301", "city" : "BRATTLEBORO", "loc" : [ -72.593322, 42.857353 ], "pop" : 17522, "state" : "VT" }
+{ "_id" : "05401", "city" : "BURLINGTON", "loc" : [ -73.219875, 44.484023 ], "pop" : 39127, "state" : "VT" }
+{ "_id" : "05701", "city" : "RUTLAND", "loc" : [ -72.97077299999999, 43.614131 ], "pop" : 22576, "state" : "VT" }
+{ "_id" : "06010", "city" : "BRISTOL", "loc" : [ -72.930193, 41.682293 ], "pop" : 60670, "state" : "CT" }
+{ "_id" : "06450", "city" : "MERIDEN", "loc" : [ -72.799734, 41.533396 ], "pop" : 59441, "state" : "CT" }
+{ "_id" : "06902", "city" : "STAMFORD", "loc" : [ -73.53742800000001, 41.052552 ], "pop" : 54605, "state" : "CT" }
+{ "_id" : "07002", "city" : "BAYONNE", "loc" : [ -74.119169, 40.666399 ], "pop" : 61444, "state" : "NJ" }
+{ "_id" : "07087", "city" : "WEEHAWKEN", "loc" : [ -74.030558, 40.768153 ], "pop" : 69646, "state" : "NJ" }
+{ "_id" : "07111", "city" : "IRVINGTON", "loc" : [ -74.23127100000001, 40.7261 ], "pop" : 60986, "state" : "NJ" }
+{ "_id" : "10021", "city" : "NEW YORK", "loc" : [ -73.958805, 40.768476 ], "pop" : 106564, "state" : "NY" }
+{ "_id" : "11226", "city" : "BROOKLYN", "loc" : [ -73.956985, 40.646694 ], "pop" : 111396, "state" : "NY" }
+{ "_id" : "11373", "city" : "JACKSON HEIGHTS", "loc" : [ -73.878551, 40.740388 ], "pop" : 88241, "state" : "NY" }
+{ "_id" : "17042", "city" : "CLEONA", "loc" : [ -76.425895, 40.335912 ], "pop" : 61993, "state" : "PA" }
+{ "_id" : "18042", "city" : "FORKS TOWNSHIP", "loc" : [ -75.23582, 40.6867 ], "pop" : 65784, "state" : "PA" }
+{ "_id" : "19143", "city" : "PHILADELPHIA", "loc" : [ -75.228819, 39.944815 ], "pop" : 80454, "state" : "PA" }
+{ "_id" : "19711", "city" : "NEWARK", "loc" : [ -75.737534, 39.701129 ], "pop" : 50573, "state" : "DE" }
+{ "_id" : "19720", "city" : "MANOR", "loc" : [ -75.589938, 39.67703 ], "pop" : 46906, "state" : "DE" }
+{ "_id" : "19901", "city" : "DOVER", "loc" : [ -75.535983, 39.156639 ], "pop" : 46005, "state" : "DE" }
+{ "_id" : "20011", "city" : "WASHINGTON", "loc" : [ -77.020251, 38.951786 ], "pop" : 62924, "state" : "DC" }
+{ "_id" : "20301", "city" : "PENTAGON", "loc" : [ -77.038196, 38.891019 ], "pop" : 21, "state" : "DC" }
+{ "_id" : "21061", "city" : "GLEN BURNIE", "loc" : [ -76.61886199999999, 39.158968 ], "pop" : 75692, "state" : "MD" }
+{ "_id" : "21207", "city" : "GWYNN OAK", "loc" : [ -76.734064, 39.329628 ], "pop" : 76002, "state" : "MD" }
+{ "_id" : "21215", "city" : "BALTIMORE", "loc" : [ -76.67939699999999, 39.344572 ], "pop" : 74402, "state" : "MD" }
+{ "_id" : "22901", "city" : "CHARLOTTESVILLE", "loc" : [ -78.490869, 38.054752 ], "pop" : 62708, "state" : "VA" }
+{ "_id" : "23464", "city" : "VIRGINIA BEACH", "loc" : [ -76.175909, 36.797772 ], "pop" : 67276, "state" : "VA" }
+{ "_id" : "23602", "city" : "NEWPORT NEWS", "loc" : [ -76.53212499999999, 37.131684 ], "pop" : 68525, "state" : "VA" }
+{ "_id" : "25801", "city" : "BECKLEY", "loc" : [ -81.206084, 37.793214 ], "pop" : 45196, "state" : "WV" }
+{ "_id" : "26003", "city" : "ELM GROVE", "loc" : [ -80.685126, 40.072736 ], "pop" : 49136, "state" : "WV" }
+{ "_id" : "26505", "city" : "STAR CITY", "loc" : [ -79.95422499999999, 39.633858 ], "pop" : 70185, "state" : "WV" }
+{ "_id" : "27292", "city" : "LEXINGTON", "loc" : [ -80.262049, 35.82306 ], "pop" : 69179, "state" : "NC" }
+{ "_id" : "28677", "city" : "STATESVILLE", "loc" : [ -80.894009, 35.799022 ], "pop" : 52895, "state" : "NC" }
+{ "_id" : "29150", "city" : "OSWEGO", "loc" : [ -80.32100800000001, 33.928199 ], "pop" : 46394, "state" : "SC" }
+{ "_id" : "29501", "city" : "FLORENCE", "loc" : [ -79.772786, 34.18375 ], "pop" : 66990, "state" : "SC" }
+{ "_id" : "29801", "city" : "AIKEN", "loc" : [ -81.71942900000001, 33.553024 ], "pop" : 51233, "state" : "SC" }
+{ "_id" : "30032", "city" : "DECATUR", "loc" : [ -84.263165, 33.740825 ], "pop" : 56056, "state" : "GA" }
+{ "_id" : "30906", "city" : "PEACH ORCHARD", "loc" : [ -82.038358, 33.402024 ], "pop" : 58646, "state" : "GA" }
+{ "_id" : "32216", "city" : "JACKSONVILLE", "loc" : [ -81.547387, 30.293907 ], "pop" : 58867, "state" : "FL" }
+{ "_id" : "33012", "city" : "HIALEAH", "loc" : [ -80.30589999999999, 25.865395 ], "pop" : 73194, "state" : "FL" }
+{ "_id" : "33311", "city" : "FORT LAUDERDALE", "loc" : [ -80.172786, 26.142104 ], "pop" : 65378, "state" : "FL" }
+{ "_id" : "35215", "city" : "CENTER POINT", "loc" : [ -86.693197, 33.635447 ], "pop" : 43862, "state" : "AL" }
+{ "_id" : "35401", "city" : "TUSCALOOSA", "loc" : [ -87.56266599999999, 33.196891 ], "pop" : 42124, "state" : "AL" }
+{ "_id" : "35901", "city" : "SOUTHSIDE", "loc" : [ -86.010279, 33.997248 ], "pop" : 44165, "state" : "AL" }
+{ "_id" : "37042", "city" : "CLARKSVILLE", "loc" : [ -87.418621, 36.585315 ], "pop" : 43296, "state" : "TN" }
+{ "_id" : "37211", "city" : "NASHVILLE", "loc" : [ -86.72403799999999, 36.072486 ], "pop" : 51478, "state" : "TN" }
+{ "_id" : "38109", "city" : "MEMPHIS", "loc" : [ -90.073238, 35.042538 ], "pop" : 60508, "state" : "TN" }
+{ "_id" : "39180", "city" : "VICKSBURG", "loc" : [ -90.85065, 32.325824 ], "pop" : 46968, "state" : "MS" }
+{ "_id" : "39401", "city" : "HATTIESBURG", "loc" : [ -89.306471, 31.314553 ], "pop" : 41866, "state" : "MS" }
+{ "_id" : "39440", "city" : "LAUREL", "loc" : [ -89.13115500000001, 31.705444 ], "pop" : 45040, "state" : "MS" }
+{ "_id" : "40214", "city" : "LOUISVILLE", "loc" : [ -85.77802699999999, 38.159318 ], "pop" : 42198, "state" : "KY" }
+{ "_id" : "40216", "city" : "SHIVELY", "loc" : [ -85.831771, 38.186138 ], "pop" : 41719, "state" : "KY" }
+{ "_id" : "40601", "city" : "HATTON", "loc" : [ -84.88061, 38.192831 ], "pop" : 46563, "state" : "KY" }
+{ "_id" : "44035", "city" : "ELYRIA", "loc" : [ -82.10508799999999, 41.372353 ], "pop" : 66674, "state" : "OH" }
+{ "_id" : "44060", "city" : "MENTOR", "loc" : [ -81.342133, 41.689468 ], "pop" : 60109, "state" : "OH" }
+{ "_id" : "44107", "city" : "EDGEWATER", "loc" : [ -81.79714300000001, 41.482654 ], "pop" : 59702, "state" : "OH" }
+{ "_id" : "46360", "city" : "MICHIGAN CITY", "loc" : [ -86.869899, 41.698031 ], "pop" : 55392, "state" : "IN" }
+{ "_id" : "47130", "city" : "JEFFERSONVILLE", "loc" : [ -85.735885, 38.307767 ], "pop" : 56543, "state" : "IN" }
+{ "_id" : "47906", "city" : "WEST LAFAYETTE", "loc" : [ -86.923661, 40.444025 ], "pop" : 54702, "state" : "IN" }
+{ "_id" : "48180", "city" : "TAYLOR", "loc" : [ -83.267269, 42.231738 ], "pop" : 70811, "state" : "MI" }
+{ "_id" : "48185", "city" : "WESTLAND", "loc" : [ -83.374908, 42.318882 ], "pop" : 84712, "state" : "MI" }
+{ "_id" : "48227", "city" : "DETROIT", "loc" : [ -83.193732, 42.388303 ], "pop" : 68390, "state" : "MI" }
+{ "_id" : "50010", "city" : "AMES", "loc" : [ -93.639398, 42.029859 ], "pop" : 52105, "state" : "IA" }
+{ "_id" : "50317", "city" : "PLEASANT HILL", "loc" : [ -93.549446, 41.612499 ], "pop" : 39883, "state" : "IA" }
+{ "_id" : "52001", "city" : "DUBUQUE", "loc" : [ -90.68191400000001, 42.514977 ], "pop" : 41934, "state" : "IA" }
+{ "_id" : "53209", "city" : "MILWAUKEE", "loc" : [ -87.947834, 43.118765 ], "pop" : 51008, "state" : "WI" }
+{ "_id" : "54401", "city" : "WAUSAU", "loc" : [ -89.633955, 44.963433 ], "pop" : 51083, "state" : "WI" }
+{ "_id" : "54901", "city" : "OSHKOSH", "loc" : [ -88.54363499999999, 44.021962 ], "pop" : 57187, "state" : "WI" }
+{ "_id" : "55106", "city" : "SAINT PAUL", "loc" : [ -93.048817, 44.968384 ], "pop" : 47905, "state" : "MN" }
+{ "_id" : "55112", "city" : "NEW BRIGHTON", "loc" : [ -93.199691, 45.074129 ], "pop" : 44128, "state" : "MN" }
+{ "_id" : "55337", "city" : "BURNSVILLE", "loc" : [ -93.275283, 44.76086 ], "pop" : 51421, "state" : "MN" }
+{ "_id" : "57103", "city" : "SIOUX FALLS", "loc" : [ -96.686415, 43.537386 ], "pop" : 32508, "state" : "SD" }
+{ "_id" : "57401", "city" : "ABERDEEN", "loc" : [ -98.485642, 45.466109 ], "pop" : 28786, "state" : "SD" }
+{ "_id" : "57701", "city" : "ROCKERVILLE", "loc" : [ -103.200259, 44.077041 ], "pop" : 45328, "state" : "SD" }
+{ "_id" : "58103", "city" : "FARGO", "loc" : [ -96.812252, 46.856406 ], "pop" : 38483, "state" : "ND" }
+{ "_id" : "58501", "city" : "BISMARCK", "loc" : [ -100.774755, 46.823448 ], "pop" : 36602, "state" : "ND" }
+{ "_id" : "58701", "city" : "MINOT", "loc" : [ -101.298476, 48.22914 ], "pop" : 42195, "state" : "ND" }
+{ "_id" : "59102", "city" : "BILLINGS", "loc" : [ -108.572662, 45.781265 ], "pop" : 40121, "state" : "MT" }
+{ "_id" : "59601", "city" : "HELENA", "loc" : [ -112.021283, 46.613066 ], "pop" : 40102, "state" : "MT" }
+{ "_id" : "59801", "city" : "MISSOULA", "loc" : [ -114.025207, 46.856274 ], "pop" : 33811, "state" : "MT" }
+{ "_id" : "60623", "city" : "CHICAGO", "loc" : [ -87.7157, 41.849015 ], "pop" : 112047, "state" : "IL" }
+{ "_id" : "60634", "city" : "NORRIDGE", "loc" : [ -87.796054, 41.945213 ], "pop" : 69160, "state" : "IL" }
+{ "_id" : "60650", "city" : "CICERO", "loc" : [ -87.76008, 41.84776 ], "pop" : 67670, "state" : "IL" }
+{ "_id" : "63031", "city" : "FLORISSANT", "loc" : [ -90.340097, 38.806865 ], "pop" : 52659, "state" : "MO" }
+{ "_id" : "63116", "city" : "SAINT LOUIS", "loc" : [ -90.26254299999999, 38.581356 ], "pop" : 49014, "state" : "MO" }
+{ "_id" : "63136", "city" : "JENNINGS", "loc" : [ -90.260189, 38.738878 ], "pop" : 54994, "state" : "MO" }
+{ "_id" : "66502", "city" : "MANHATTAN", "loc" : [ -96.585776, 39.193757 ], "pop" : 50178, "state" : "KS" }
+{ "_id" : "67212", "city" : "WICHITA", "loc" : [ -97.438344, 37.700683 ], "pop" : 41349, "state" : "KS" }
+{ "_id" : "67401", "city" : "BAVARIA", "loc" : [ -97.60878700000001, 38.823802 ], "pop" : 45208, "state" : "KS" }
+{ "_id" : "68104", "city" : "OMAHA", "loc" : [ -95.999888, 41.29186 ], "pop" : 35325, "state" : "NE" }
+{ "_id" : "68502", "city" : "LINCOLN", "loc" : [ -96.693763, 40.789282 ], "pop" : 27576, "state" : "NE" }
+{ "_id" : "68847", "city" : "KEARNEY", "loc" : [ -99.077883, 40.713608 ], "pop" : 28674, "state" : "NE" }
+{ "_id" : "70072", "city" : "MARRERO", "loc" : [ -90.110462, 29.859756 ], "pop" : 58905, "state" : "LA" }
+{ "_id" : "70117", "city" : "NEW ORLEANS", "loc" : [ -90.03124, 29.970298 ], "pop" : 56494, "state" : "LA" }
+{ "_id" : "70560", "city" : "NEW IBERIA", "loc" : [ -91.819959, 30.001027 ], "pop" : 56105, "state" : "LA" }
+{ "_id" : "72032", "city" : "CONWAY", "loc" : [ -92.423574, 35.084199 ], "pop" : 43236, "state" : "AR" }
+{ "_id" : "72076", "city" : "GRAVEL RIDGE", "loc" : [ -92.13043500000001, 34.881985 ], "pop" : 37428, "state" : "AR" }
+{ "_id" : "72401", "city" : "JONESBORO", "loc" : [ -90.69652600000001, 35.833016 ], "pop" : 53532, "state" : "AR" }
+{ "_id" : "73034", "city" : "EDMOND", "loc" : [ -97.47983499999999, 35.666483 ], "pop" : 43814, "state" : "OK" }
+{ "_id" : "73505", "city" : "LAWTON", "loc" : [ -98.455234, 34.617939 ], "pop" : 45542, "state" : "OK" }
+{ "_id" : "74801", "city" : "SHAWNEE", "loc" : [ -96.931321, 35.34907 ], "pop" : 40076, "state" : "OK" }
+{ "_id" : "78207", "city" : "SAN ANTONIO", "loc" : [ -98.52596699999999, 29.422855 ], "pop" : 58355, "state" : "TX" }
+{ "_id" : "78521", "city" : "BROWNSVILLE", "loc" : [ -97.461236, 25.922103 ], "pop" : 79463, "state" : "TX" }
+{ "_id" : "78572", "city" : "ALTON", "loc" : [ -98.342647, 26.24153 ], "pop" : 67604, "state" : "TX" }
+{ "_id" : "80123", "city" : "BOW MAR", "loc" : [ -105.07766, 39.596854 ], "pop" : 59418, "state" : "CO" }
+{ "_id" : "80221", "city" : "FEDERAL HEIGHTS", "loc" : [ -105.007985, 39.840562 ], "pop" : 54069, "state" : "CO" }
+{ "_id" : "80631", "city" : "GARDEN CITY", "loc" : [ -104.704756, 40.413968 ], "pop" : 53905, "state" : "CO" }
+{ "_id" : "82001", "city" : "CHEYENNE", "loc" : [ -104.796234, 41.143719 ], "pop" : 33107, "state" : "WY" }
+{ "_id" : "82070", "city" : "LARAMIE", "loc" : [ -105.581146, 41.312907 ], "pop" : 29327, "state" : "WY" }
+{ "_id" : "82716", "city" : "GILLETTE", "loc" : [ -105.497442, 44.282009 ], "pop" : 25968, "state" : "WY" }
+{ "_id" : "83301", "city" : "TWIN FALLS", "loc" : [ -114.469265, 42.556495 ], "pop" : 34539, "state" : "ID" }
+{ "_id" : "83704", "city" : "BOISE", "loc" : [ -116.295099, 43.633001 ], "pop" : 40912, "state" : "ID" }
+{ "_id" : "83814", "city" : "COEUR D ALENE", "loc" : [ -116.784976, 47.692841 ], "pop" : 33589, "state" : "ID" }
+{ "_id" : "84118", "city" : "KEARNS", "loc" : [ -111.98521, 40.652759 ], "pop" : 55999, "state" : "UT" }
+{ "_id" : "84120", "city" : "WEST VALLEY CITY", "loc" : [ -112.009783, 40.68708 ], "pop" : 52854, "state" : "UT" }
+{ "_id" : "84604", "city" : "PROVO", "loc" : [ -111.654906, 40.260681 ], "pop" : 43841, "state" : "UT" }
+{ "_id" : "85023", "city" : "PHOENIX", "loc" : [ -112.111838, 33.632383 ], "pop" : 54668, "state" : "AZ" }
+{ "_id" : "85204", "city" : "MESA", "loc" : [ -111.789554, 33.399168 ], "pop" : 55180, "state" : "AZ" }
+{ "_id" : "85364", "city" : "YUMA", "loc" : [ -114.642362, 32.701507 ], "pop" : 57131, "state" : "AZ" }
+{ "_id" : "87501", "city" : "POJOAQUE VALLEY", "loc" : [ -105.974818, 35.702472 ], "pop" : 51715, "state" : "NM" }
+{ "_id" : "88001", "city" : "LAS CRUCES", "loc" : [ -106.746034, 32.321641 ], "pop" : 57502, "state" : "NM" }
+{ "_id" : "88201", "city" : "ROSWELL", "loc" : [ -104.525857, 33.388504 ], "pop" : 53644, "state" : "NM" }
+{ "_id" : "89031", "city" : "NORTH LAS VEGAS", "loc" : [ -115.124832, 36.206228 ], "pop" : 48113, "state" : "NV" }
+{ "_id" : "89115", "city" : "LAS VEGAS", "loc" : [ -115.067062, 36.215818 ], "pop" : 51532, "state" : "NV" }
+{ "_id" : "89502", "city" : "RENO", "loc" : [ -119.776395, 39.497239 ], "pop" : 38332, "state" : "NV" }
+{ "_id" : "90011", "city" : "LOS ANGELES", "loc" : [ -118.258189, 34.007856 ], "pop" : 96074, "state" : "CA" }
+{ "_id" : "90201", "city" : "BELL GARDENS", "loc" : [ -118.17205, 33.969177 ], "pop" : 99568, "state" : "CA" }
+{ "_id" : "90650", "city" : "NORWALK", "loc" : [ -118.081767, 33.90564 ], "pop" : 94188, "state" : "CA" }
+{ "_id" : "96734", "city" : "KAILUA", "loc" : [ -157.744781, 21.406262 ], "pop" : 53403, "state" : "HI" }
+{ "_id" : "96744", "city" : "KANEOHE", "loc" : [ -157.811543, 21.422819 ], "pop" : 55236, "state" : "HI" }
+{ "_id" : "96818", "city" : "HONOLULU", "loc" : [ -157.926925, 21.353173 ], "pop" : 62915, "state" : "HI" }
+{ "_id" : "97005", "city" : "BEAVERTON", "loc" : [ -122.805395, 45.475035 ], "pop" : 46660, "state" : "OR" }
+{ "_id" : "97206", "city" : "PORTLAND", "loc" : [ -122.59727, 45.483995 ], "pop" : 43134, "state" : "OR" }
+{ "_id" : "97301", "city" : "SALEM", "loc" : [ -122.979692, 44.926039 ], "pop" : 48007, "state" : "OR" }
+{ "_id" : "98031", "city" : "KENT", "loc" : [ -122.193184, 47.388004 ], "pop" : 50515, "state" : "WA" }
+{ "_id" : "98059", "city" : "RENTON", "loc" : [ -122.151178, 47.467383 ], "pop" : 48197, "state" : "WA" }
+{ "_id" : "98310", "city" : "BREMERTON", "loc" : [ -122.629913, 47.601916 ], "pop" : 49057, "state" : "WA" }
+{ "_id" : "99504", "city" : "ANCHORAGE", "loc" : [ -149.74467, 61.203696 ], "pop" : 32383, "state" : "AK" }
+{ "_id" : "99709", "city" : "FAIRBANKS", "loc" : [ -147.846917, 64.85437 ], "pop" : 23238, "state" : "AK" }
+{ "_id" : "99801", "city" : "JUNEAU", "loc" : [ -134.529429, 58.362767 ], "pop" : 24947, "state" : "AK" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch5/pom.xml
----------------------------------------------------------------------
diff --git a/elasticsearch5/pom.xml b/elasticsearch5/pom.xml
index 21d60f1..0fe9df9 100644
--- a/elasticsearch5/pom.xml
+++ b/elasticsearch5/pom.xml
@@ -78,6 +78,35 @@ limitations under the License.
       <version>${elasticsearch5-java-driver.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.elasticsearch.plugin</groupId>
+      <artifactId>transport-netty3-client</artifactId>
+      <version>${elasticsearch5-java-driver.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <!-- Elastic search doesn't export painless script artifact to maven central.
+      Using 3rd party version (codelibs)
+      This JAR is used only in tests
+      https://discuss.elastic.co/t/5-0-0-using-painless-in-esintegtestcase/64447/13
+      -->
+      <groupId>org.codelibs.elasticsearch.module</groupId>
+      <artifactId>lang-painless</artifactId>
+      <version>${elasticsearch5-java-driver.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+      <version>${log4j2.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-core</artifactId>
+      <version>${log4j2.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>com.carrotsearch</groupId>
       <artifactId>hppc</artifactId>
       <version>${hppc.version}</version>
@@ -110,6 +139,8 @@ limitations under the License.
                 <ignoredUnusedDeclaredDependency>org.apache.calcite.avatica:avatica</ignoredUnusedDeclaredDependency>
                 <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-api</ignoredUnusedDeclaredDependency>
                 <ignoredUnusedDeclaredDependency>org.slf4j:slf4j-log4j12</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.apache.logging.log4j:log4j-api</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.apache.logging.log4j:log4j-core</ignoredUnusedDeclaredDependency>
               </ignoredUnusedDeclaredDependencies>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
index 6e16f2f..ebb96c8 100644
--- a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Schema.java
@@ -22,6 +22,8 @@ import org.apache.calcite.schema.impl.AbstractSchema;
 
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
@@ -87,6 +89,17 @@ public class Elasticsearch5Schema extends AbstractSchema
     }
   }
 
+  /**
+   * Allows schema to be instantiated from existing elastic search client.
+   * This constructor is used in tests.
+   */
+  @VisibleForTesting
+  Elasticsearch5Schema(Client client, String index) {
+    this.client = Preconditions.checkNotNull(client, "client");
+    this.index = Preconditions.checkNotNull(index, "index");
+  }
+
+
   @Override protected Map<String, Table> getTableMap() {
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
 
@@ -123,8 +136,10 @@ public class Elasticsearch5Schema extends AbstractSchema
 
     final List<DiscoveryNode> nodes =
         ImmutableList.copyOf(transportClient.connectedNodes());
+
     if (nodes.isEmpty()) {
-      throw new RuntimeException("Cannot connect to any elasticsearch nodes");
+      throw new IllegalStateException("Cannot connect to any elasticsearch node from "
+              + transportNodes);
     }
 
     client = transportClient;

http://git-wip-us.apache.org/repos/asf/calcite/blob/e7b353bb/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
index 175e3cc..66e3edf 100644
--- a/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
+++ b/elasticsearch5/src/main/java/org/apache/calcite/adapter/elasticsearch5/Elasticsearch5Table.java
@@ -25,15 +25,19 @@ import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.util.Util;
 
 import org.elasticsearch.client.Client;
+import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.NamedXContentRegistry;
 import org.elasticsearch.common.xcontent.XContent;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.SearchModule;
 import org.elasticsearch.search.builder.SearchSourceBuilder;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -61,13 +65,16 @@ public class Elasticsearch5Table extends AbstractElasticsearchTable {
       searchSourceBuilder = new SearchSourceBuilder();
     } else {
       String queryString = "{" + Util.toString(ops, "", ", ", "") + "}";
-      NamedXContentRegistry xContentRegistry = NamedXContentRegistry.EMPTY;
       XContent xContent = JsonXContent.jsonXContent;
+      SearchModule searchModule = new SearchModule(Settings.EMPTY, false, Collections.emptyList());
+      NamedXContentRegistry xContentRegistry =
+              new NamedXContentRegistry(searchModule.getNamedXContents());
+
       try (XContentParser parser = xContent.createParser(xContentRegistry, queryString)) {
         final QueryParseContext queryParseContext = new QueryParseContext(parser);
         searchSourceBuilder = SearchSourceBuilder.fromXContent(queryParseContext);
       } catch (IOException ex) {
-        throw new RuntimeException(ex);
+        throw new UncheckedIOException(ex);
       }
     }
     final Function1<SearchHit, Object> getter = Elasticsearch5Enumerator.getter(fields);