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/07/28 01:32:26 UTC

[3/3] calcite git commit: [CALCITE-2419] Use embedded Cassandra for tests

[CALCITE-2419] Use embedded Cassandra for tests

Leverage cassandra-unit library to start a cassandra mini cluster. Database is being populated from
local twissandra.cql file. All other configuration files are also located in test classpath.

Contratry to default project settings (for failsafe and surefire plugins), enforce tests to run sequentially
(using @NotThreadSafe annotation). Some static variables (eg. Hook(s)) are being shared causing
non-deterministic failures (due to concurrent access). It is done to avoid flaky tests.
This limitation should be addressed in future.

Disable adapter test on JDK11 which is not supported by Cassandra 3.x (4.x is still in development).

Close apache/calcite#764
Close apache/calcite#769


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

Branch: refs/heads/master
Commit: bd0e140026c409aa932cebf0e1fe223ab3a5319a
Parents: 3c40d86
Author: Michael Mior <mm...@uwaterloo.ca>
Authored: Wed Jul 18 10:27:02 2018 -0400
Committer: Michael Mior <mm...@uwaterloo.ca>
Committed: Fri Jul 27 21:31:19 2018 -0400

----------------------------------------------------------------------
 cassandra/pom.xml                               |  14 +-
 .../adapter/cassandra/CassandraSchema.java      |  37 +-
 .../cassandra/CassandraSchemaFactory.java       |   8 +-
 .../apache/calcite/test/CassandraAdapterIT.java | 171 -----
 .../calcite/test/CassandraAdapterTest.java      | 220 ++++++
 cassandra/src/test/resources/cassandra.yaml     | 600 +++++++++++++++++
 cassandra/src/test/resources/logback-test.xml   |  32 +
 cassandra/src/test/resources/model.json         |   1 +
 cassandra/src/test/resources/twissandra.cql     | 663 +++++++++++++++++++
 pom.xml                                         |  13 +
 src/main/config/checkstyle/suppressions.xml     |   2 +-
 11 files changed, 1583 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/cassandra/pom.xml b/cassandra/pom.xml
index 8d8e618..3cbee66 100644
--- a/cassandra/pom.xml
+++ b/cassandra/pom.xml
@@ -70,12 +70,22 @@ limitations under the License.
       <artifactId>cassandra-driver-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.cassandraunit</groupId>
+      <artifactId>cassandra-unit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.cassandra</groupId>
+      <artifactId>cassandra-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
+      <groupId>com.github.stephenc.jcip</groupId>
+      <artifactId>jcip-annotations</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
----------------------------------------------------------------------
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
index f2bcdcd..eac7002 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchema.java
@@ -56,6 +56,7 @@ import org.slf4j.Logger;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -72,6 +73,8 @@ public class CassandraSchema extends AbstractSchema {
 
   protected static final Logger LOGGER = CalciteTrace.getPlannerTracer();
 
+  private static final int DEFAULT_CASSANDRA_PORT = 9042;
+
   /**
    * Creates a Cassandra schema.
    *
@@ -79,7 +82,19 @@ public class CassandraSchema extends AbstractSchema {
    * @param keyspace Cassandra keyspace name, e.g. "twissandra"
    */
   public CassandraSchema(String host, String keyspace, SchemaPlus parentSchema, String name) {
-    this(host, keyspace, null, null, parentSchema, name);
+    this(host, DEFAULT_CASSANDRA_PORT, keyspace, null, null, parentSchema, name);
+  }
+
+  /**
+   * Creates a Cassandra schema.
+   *
+   * @param host Cassandra host, e.g. "localhost"
+   * @param port Cassandra port, e.g. 9042
+   * @param keyspace Cassandra keyspace name, e.g. "twissandra"
+   */
+  public CassandraSchema(String host, int port, String keyspace,
+          SchemaPlus parentSchema, String name) {
+    this(host, port, keyspace, null, null, parentSchema, name);
   }
 
   /**
@@ -92,16 +107,32 @@ public class CassandraSchema extends AbstractSchema {
    */
   public CassandraSchema(String host, String keyspace, String username, String password,
         SchemaPlus parentSchema, String name) {
+    this(host, DEFAULT_CASSANDRA_PORT, keyspace, null, null, parentSchema, name);
+  }
+
+  /**
+   * Creates a Cassandra schema.
+   *
+   * @param host Cassandra host, e.g. "localhost"
+   * @param port Cassandra port, e.g. 9042
+   * @param keyspace Cassandra keyspace name, e.g. "twissandra"
+   * @param username Cassandra username
+   * @param password Cassandra password
+   */
+  public CassandraSchema(String host, int port, String keyspace, String username, String password,
+        SchemaPlus parentSchema, String name) {
     super();
 
     this.keyspace = keyspace;
     try {
       Cluster cluster;
+      List<InetSocketAddress> contactPoints = new ArrayList<>(1);
+      contactPoints.add(new InetSocketAddress(host, port));
       if (username != null && password != null) {
-        cluster = Cluster.builder().addContactPoint(host)
+        cluster = Cluster.builder().addContactPointsWithPorts(contactPoints)
             .withCredentials(username, password).build();
       } else {
-        cluster = Cluster.builder().addContactPoint(host).build();
+        cluster = Cluster.builder().addContactPointsWithPorts(contactPoints).build();
       }
 
       this.session = cluster.connect(keyspace);

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
----------------------------------------------------------------------
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
index 136b81b..3b2ab96 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraSchemaFactory.java
@@ -37,7 +37,13 @@ public class CassandraSchemaFactory implements SchemaFactory {
     String keyspace = (String) map.get("keyspace");
     String username = (String) map.get("username");
     String password = (String) map.get("password");
-    return new CassandraSchema(host, keyspace, username, password, parentSchema, name);
+
+    if (map.containsKey("port")) {
+      int port = (int) map.get("port");
+      return new CassandraSchema(host, port, keyspace, username, password, parentSchema, name);
+    } else {
+      return new CassandraSchema(host, keyspace, username, password, parentSchema, name);
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterIT.java
----------------------------------------------------------------------
diff --git a/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterIT.java b/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterIT.java
deleted file mode 100644
index ffb0010..0000000
--- a/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterIT.java
+++ /dev/null
@@ -1,171 +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.collect.ImmutableMap;
-
-import org.junit.Test;
-
-/**
- * Tests for the {@code org.apache.calcite.adapter.cassandra} package.
- *
- * <p>Before calling this test, you need to populate Cassandra, 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 Cassandra and the "twissandra"
- * test data set.
- */
-public class CassandraAdapterIT {
-  /** Connection factory based on the "mongo-zips" model. */
-  public static final ImmutableMap<String, String> TWISSANDRA =
-      ImmutableMap.of("model",
-          CassandraAdapterIT.class.getResource("/model.json")
-              .getPath());
-
-  /** Whether to run Cassandra tests. Enabled by default, however test is only
-   * included if "it" profile is activated ({@code -Pit}). To disable,
-   * specify {@code -Dcalcite.test.cassandra=false} on the Java command line. */
-  public static final boolean ENABLED =
-      Util.getBooleanProperty("calcite.test.cassandra", true);
-
-  /** Whether to run this test. */
-  protected boolean enabled() {
-    return ENABLED;
-  }
-
-  @Test public void testSelect() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select * from \"users\"")
-        .returnsCount(10);
-  }
-
-  @Test public void testFilter() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select * from \"userline\" where \"username\"='!PUBLIC!'")
-        .limit(1)
-        .returns("username=!PUBLIC!; time=e8754000-80b8-1fe9-8e73-e3698c967ddd; "
-            + "tweet_id=f3c329de-d05b-11e5-b58b-90e2ba530b12\n")
-        .explainContains("PLAN=CassandraToEnumerableConverter\n"
-           + "  CassandraFilter(condition=[=($0, '!PUBLIC!')])\n"
-           + "    CassandraTableScan(table=[[twissandra, userline]]");
-  }
-
-  @Test public void testFilterUUID() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select * from \"tweets\" where \"tweet_id\"='f3cd759c-d05b-11e5-b58b-90e2ba530b12'")
-        .limit(1)
-        .returns("tweet_id=f3cd759c-d05b-11e5-b58b-90e2ba530b12; "
-            + "body=Lacus augue pede posuere.; username=JmuhsAaMdw\n")
-        .explainContains("PLAN=CassandraToEnumerableConverter\n"
-           + "  CassandraFilter(condition=[=(CAST($0):CHAR(36) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'f3cd759c-d05b-11e5-b58b-90e2ba530b12')])\n"
-           + "    CassandraTableScan(table=[[twissandra, tweets]]");
-  }
-
-  @Test public void testSort() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select * from \"userline\" where \"username\" = '!PUBLIC!' order by \"time\" desc")
-        .returnsCount(146)
-        .explainContains("PLAN=CassandraToEnumerableConverter\n"
-            + "  CassandraSort(sort0=[$1], dir0=[DESC])\n"
-            + "    CassandraFilter(condition=[=($0, '!PUBLIC!')])\n");
-  }
-
-  @Test public void testProject() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select \"tweet_id\" from \"userline\" where \"username\" = '!PUBLIC!' limit 2")
-        .returns("tweet_id=f3c329de-d05b-11e5-b58b-90e2ba530b12\n"
-               + "tweet_id=f3dbb03a-d05b-11e5-b58b-90e2ba530b12\n")
-        .explainContains("PLAN=CassandraToEnumerableConverter\n"
-                + "  CassandraLimit(fetch=[2])\n"
-                + "    CassandraProject(tweet_id=[$2])\n"
-                + "      CassandraFilter(condition=[=($0, '!PUBLIC!')])\n");
-  }
-
-  @Test public void testProjectAlias() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select \"tweet_id\" as \"foo\" from \"userline\" "
-                + "where \"username\" = '!PUBLIC!' limit 1")
-        .returns("foo=f3c329de-d05b-11e5-b58b-90e2ba530b12\n");
-  }
-
-  @Test public void testProjectConstant() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select 'foo' as \"bar\" from \"userline\" limit 1")
-        .returns("bar=foo\n");
-  }
-
-  @Test public void testLimit() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select \"tweet_id\" from \"userline\" where \"username\" = '!PUBLIC!' limit 8")
-        .explainContains("CassandraLimit(fetch=[8])\n");
-  }
-
-  @Test public void testSortLimit() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select * from \"userline\" where \"username\"='!PUBLIC!' "
-             + "order by \"time\" desc limit 10")
-        .explainContains("  CassandraLimit(fetch=[10])\n"
-                       + "    CassandraSort(sort0=[$1], dir0=[DESC])");
-  }
-
-  @Test public void testSortOffset() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select \"tweet_id\" from \"userline\" where "
-             + "\"username\"='!PUBLIC!' limit 2 offset 1")
-        .explainContains("CassandraLimit(offset=[1], fetch=[2])")
-        .returns("tweet_id=f3dbb03a-d05b-11e5-b58b-90e2ba530b12\n"
-               + "tweet_id=f3e4182e-d05b-11e5-b58b-90e2ba530b12\n");
-  }
-
-  @Test public void testMaterializedView() {
-    CalciteAssert.that()
-        .enable(enabled())
-        .with(TWISSANDRA)
-        .query("select \"tweet_id\" from \"tweets\" where \"username\"='JmuhsAaMdw'")
-        .enableMaterializations(true)
-        .explainContains("CassandraTableScan(table=[[twissandra, tweets_by_user]])");
-  }
-}
-
-// End CassandraAdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
----------------------------------------------------------------------
diff --git a/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java b/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
new file mode 100644
index 0000000..adc5d90
--- /dev/null
+++ b/cassandra/src/test/java/org/apache/calcite/test/CassandraAdapterTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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 org.apache.cassandra.config.DatabaseDescriptor;
+
+import com.google.common.collect.ImmutableMap;
+
+import net.jcip.annotations.NotThreadSafe;
+
+import org.cassandraunit.CassandraCQLUnit;
+import org.cassandraunit.dataset.cql.ClassPathCQLDataSet;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Tests for the {@code org.apache.calcite.adapter.cassandra} package.
+ *
+ * <p>Will start embedded cassandra cluster and populate it from local {@code twissandra.cql} file.
+ * All configuration files are located in test classpath.
+ *
+ * <p>Note that tests will be skipped if running on JDK11 (which is not yet supported by cassandra)
+ * see <a href="https://issues.apache.org/jira/browse/CASSANDRA-9608">CASSANDRA-9608</a>.
+ *
+ */
+// force tests to run sequentially (maven surefire and failsafe are running them in parallel)
+// seems like some of our code is sharing static variables (like Hooks) which causes tests
+// to fail non-deterministically (flaky tests).
+@NotThreadSafe
+public class CassandraAdapterTest {
+
+  @ClassRule
+  public static final ExternalResource RULE = initCassandraIfEnabled();
+
+  /** Connection factory based on the "mongo-zips" model. */
+  private static final ImmutableMap<String, String> TWISSANDRA =
+      ImmutableMap.of("model",
+          CassandraAdapterTest.class.getResource("/model.json")
+              .getPath());
+
+  /**
+   * Whether to run this test.
+   * <p>Enabled by default, unless explicitly disabled
+   * from command line ({@code -Dcalcite.test.cassandra=false}) or running on incompatible JDK
+   * version (see below).
+   *
+   * <p>As of this wiring Cassandra 4.x is not yet released and we're using 3.x
+   * (which fails on JDK11). All cassandra tests will be skipped if running on JDK11.
+   * TODO: remove JDK check once current adapter supports Cassandra 4.x
+   *
+   * @see <a href="https://issues.apache.org/jira/browse/CASSANDRA-9608">CASSANDRA-9608</a>
+   * @return {@code true} if test is compatible with current environment,
+   *         {@code false} otherwise
+   */
+  private static boolean enabled() {
+    final boolean enabled =
+        Util.getBooleanProperty("calcite.test.cassandra", true);
+    final int major = Integer.parseInt(System.getProperty("java.version").split("\\.")[0]);
+    final boolean compatibleJdk = major != 11;
+
+    return enabled && compatibleJdk;
+  }
+
+  private static ExternalResource initCassandraIfEnabled() {
+    if (!enabled()) {
+      // Return NOP resource (to avoid nulls)
+      return new ExternalResource() {
+        @Override public Statement apply(final Statement base, final Description description) {
+          return super.apply(base, description);
+        }
+      };
+    }
+
+    CassandraCQLUnit rule = new CassandraCQLUnit(
+        new ClassPathCQLDataSet("twissandra.cql"));
+
+    // This static init is necessary otherwise tests fail with CassandraUnit in IntelliJ (jdk10)
+    // should be called right after constructor
+    // NullPointerException for DatabaseDescriptor.getDiskFailurePolicy
+    // for more info see
+    // https://github.com/jsevellec/cassandra-unit/issues/249
+    // https://github.com/jsevellec/cassandra-unit/issues/221
+    DatabaseDescriptor.daemonInitialization();
+
+    return rule;
+  }
+
+  @BeforeClass
+  public static void setUp() {
+    // run tests only if explicitly enabled
+    assumeTrue("test explicitly disabled", enabled());
+  }
+
+  @Test public void testSelect() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select * from \"users\"")
+        .returnsCount(10);
+  }
+
+  @Test public void testFilter() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select * from \"userline\" where \"username\"='!PUBLIC!'")
+        .limit(1)
+        .returns("username=!PUBLIC!; time=e8754000-80b8-1fe9-8e73-e3698c967ddd; "
+            + "tweet_id=f3c329de-d05b-11e5-b58b-90e2ba530b12\n")
+        .explainContains("PLAN=CassandraToEnumerableConverter\n"
+           + "  CassandraFilter(condition=[=($0, '!PUBLIC!')])\n"
+           + "    CassandraTableScan(table=[[twissandra, userline]]");
+  }
+
+  @Test public void testFilterUUID() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select * from \"tweets\" where \"tweet_id\"='f3cd759c-d05b-11e5-b58b-90e2ba530b12'")
+        .limit(1)
+        .returns("tweet_id=f3cd759c-d05b-11e5-b58b-90e2ba530b12; "
+            + "body=Lacus augue pede posuere.; username=JmuhsAaMdw\n")
+        .explainContains("PLAN=CassandraToEnumerableConverter\n"
+           + "  CassandraFilter(condition=[=(CAST($0):CHAR(36) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'f3cd759c-d05b-11e5-b58b-90e2ba530b12')])\n"
+           + "    CassandraTableScan(table=[[twissandra, tweets]]");
+  }
+
+  @Test public void testSort() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select * from \"userline\" where \"username\" = '!PUBLIC!' order by \"time\" desc")
+        .returnsCount(146)
+        .explainContains("PLAN=CassandraToEnumerableConverter\n"
+            + "  CassandraSort(sort0=[$1], dir0=[DESC])\n"
+            + "    CassandraFilter(condition=[=($0, '!PUBLIC!')])\n");
+  }
+
+  @Test public void testProject() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select \"tweet_id\" from \"userline\" where \"username\" = '!PUBLIC!' limit 2")
+        .returns("tweet_id=f3c329de-d05b-11e5-b58b-90e2ba530b12\n"
+               + "tweet_id=f3dbb03a-d05b-11e5-b58b-90e2ba530b12\n")
+        .explainContains("PLAN=CassandraToEnumerableConverter\n"
+                + "  CassandraLimit(fetch=[2])\n"
+                + "    CassandraProject(tweet_id=[$2])\n"
+                + "      CassandraFilter(condition=[=($0, '!PUBLIC!')])\n");
+  }
+
+  @Test public void testProjectAlias() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select \"tweet_id\" as \"foo\" from \"userline\" "
+                + "where \"username\" = '!PUBLIC!' limit 1")
+        .returns("foo=f3c329de-d05b-11e5-b58b-90e2ba530b12\n");
+  }
+
+  @Test public void testProjectConstant() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select 'foo' as \"bar\" from \"userline\" limit 1")
+        .returns("bar=foo\n");
+  }
+
+  @Test public void testLimit() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select \"tweet_id\" from \"userline\" where \"username\" = '!PUBLIC!' limit 8")
+        .explainContains("CassandraLimit(fetch=[8])\n");
+  }
+
+  @Test public void testSortLimit() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select * from \"userline\" where \"username\"='!PUBLIC!' "
+             + "order by \"time\" desc limit 10")
+        .explainContains("  CassandraLimit(fetch=[10])\n"
+                       + "    CassandraSort(sort0=[$1], dir0=[DESC])");
+  }
+
+  @Test public void testSortOffset() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select \"tweet_id\" from \"userline\" where "
+             + "\"username\"='!PUBLIC!' limit 2 offset 1")
+        .explainContains("CassandraLimit(offset=[1], fetch=[2])")
+        .returns("tweet_id=f3dbb03a-d05b-11e5-b58b-90e2ba530b12\n"
+               + "tweet_id=f3e4182e-d05b-11e5-b58b-90e2ba530b12\n");
+  }
+
+  @Test public void testMaterializedView() {
+    CalciteAssert.that()
+        .with(TWISSANDRA)
+        .query("select \"tweet_id\" from \"tweets\" where \"username\"='JmuhsAaMdw'")
+        .enableMaterializations(true)
+        .explainContains("CassandraTableScan(table=[[twissandra, tweets_by_user]])");
+  }
+}
+
+// End CassandraAdapterTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/test/resources/cassandra.yaml
----------------------------------------------------------------------
diff --git a/cassandra/src/test/resources/cassandra.yaml b/cassandra/src/test/resources/cassandra.yaml
new file mode 100644
index 0000000..1f3f74a
--- /dev/null
+++ b/cassandra/src/test/resources/cassandra.yaml
@@ -0,0 +1,600 @@
+# 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.
+
+# Cassandra storage config YAML
+
+# The name of the cluster. This is mainly used to prevent machines in
+# one logical cluster from joining another.
+cluster_name: 'Test Cluster'
+
+# You should always specify InitialToken when setting up a production
+# cluster for the first time, and often when adding capacity later.
+# The principle is that each node should be given an equal slice of
+# the token ring; see http://wiki.apache.org/cassandra/Operations
+# for more details.
+#
+# If blank, Cassandra will request a token bisecting the range of
+# the heaviest-loaded existing node.  If there is no load information
+# available, such as is the case with a new cluster, it will pick
+# a random token, which will lead to hot spots.
+#initial_token:
+
+# See http://wiki.apache.org/cassandra/HintedHandoff
+hinted_handoff_enabled: true
+# this defines the maximum amount of time a dead host will have hints
+# generated.  After it has been dead this long, new hints for it will not be
+# created until it has been seen alive and gone down again.
+max_hint_window_in_ms: 10800000 # 3 hours
+# Maximum throttle in KBs per second, per delivery thread.  This will be
+# reduced proportionally to the number of nodes in the cluster.  (If there
+# are two nodes in the cluster, each delivery thread will use the maximum
+# rate; if there are three, each will throttle to half of the maximum,
+# since we expect two nodes to be delivering hints simultaneously.)
+hinted_handoff_throttle_in_kb: 1024
+# Number of threads with which to deliver hints;
+# Consider increasing this number when you have multi-dc deployments, since
+# cross-dc handoff tends to be slower
+max_hints_delivery_threads: 2
+
+hints_directory: target/embeddedCassandra/hints
+
+# The following setting populates the page cache on memtable flush and compaction
+# WARNING: Enable this setting only when the whole node's data fits in memory.
+# Defaults to: false
+# populate_io_cache_on_flush: false
+
+# Authentication backend, implementing IAuthenticator; used to identify users
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator,
+# PasswordAuthenticator}.
+#
+# - AllowAllAuthenticator performs no checks - set it to disable authentication.
+# - PasswordAuthenticator relies on username/password pairs to authenticate
+#   users. It keeps usernames and hashed passwords in system_auth.credentials table.
+#   Please increase system_auth keyspace replication factor if you use this authenticator.
+authenticator: AllowAllAuthenticator
+
+# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer,
+# CassandraAuthorizer}.
+#
+# - AllowAllAuthorizer allows any action to any user - set it to disable authorization.
+# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please
+#   increase system_auth keyspace replication factor if you use this authorizer.
+authorizer: AllowAllAuthorizer
+
+# Validity period for permissions cache (fetching permissions can be an
+# expensive operation depending on the authorizer, CassandraAuthorizer is
+# one example). Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthorizer.
+permissions_validity_in_ms: 2000
+
+
+# The partitioner is responsible for distributing rows (by key) across
+# nodes in the cluster.  Any IPartitioner may be used, including your/m
+# own as long as it is on the classpath.  Out of the box, Cassandra
+# provides org.apache.cassandra.dht.{Murmur3Partitioner, RandomPartitioner
+# ByteOrderedPartitioner, OrderPreservingPartitioner (deprecated)}.
+#
+# - RandomPartitioner distributes rows across the cluster evenly by md5.
+#   This is the default prior to 1.2 and is retained for compatibility.
+# - Murmur3Partitioner is similar to RandomPartioner but uses Murmur3_128
+#   Hash Function instead of md5.  When in doubt, this is the best option.
+# - ByteOrderedPartitioner orders rows lexically by key bytes.  BOP allows
+#   scanning rows in key order, but the ordering can generate hot spots
+#   for sequential insertion workloads.
+# - OrderPreservingPartitioner is an obsolete form of BOP, that stores
+# - keys in a less-efficient format and only works with keys that are
+#   UTF8-encoded Strings.
+# - CollatingOPP collates according to EN,US rules rather than lexical byte
+#   ordering.  Use this as an example if you need custom collation.
+#
+# See http://wiki.apache.org/cassandra/Operations for more on
+# partitioners and token selection.
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+
+# directories where Cassandra should store data on disk.
+data_file_directories:
+    - target/embeddedCassandra/data
+
+# commit log
+commitlog_directory: target/embeddedCassandra/commitlog
+
+cdc_raw_directory: target/embeddedCassandra/cdc
+
+# policy for data disk failures:
+# stop: shut down gossip and Thrift, leaving the node effectively dead, but
+#       can still be inspected via JMX.
+# best_effort: stop using the failed disk and respond to requests based on
+#              remaining available sstables.  This means you WILL see obsolete
+#              data at CL.ONE!
+# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra
+disk_failure_policy: stop
+
+
+# Maximum size of the key cache in memory.
+#
+# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+# minimum, sometimes more. The key cache is fairly tiny for the amount of
+# time it saves, so it's worthwhile to use it at large numbers.
+# The row cache saves even more time, but must store the whole values of
+# its rows, so it is extremely space-intensive. It's best to only use the
+# row cache if you have hot rows or static rows.
+#
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache.
+key_cache_size_in_mb:
+
+# Duration in seconds after which Cassandra should
+# safe the keys cache. Caches are saved to saved_caches_directory as
+# specified in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 14400 or 4 hours.
+key_cache_save_period: 14400
+
+# Number of keys from the key cache to save
+# Disabled by default, meaning all keys are going to be saved
+# key_cache_keys_to_save: 100
+
+# Maximum size of the row cache in memory.
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is 0, to disable row caching.
+row_cache_size_in_mb: 0
+
+# Duration in seconds after which Cassandra should
+# safe the row cache. Caches are saved to saved_caches_directory as specified
+# in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 0 to disable saving the row cache.
+row_cache_save_period: 0
+
+# Number of keys from the row cache to save
+# Disabled by default, meaning all keys are going to be saved
+# row_cache_keys_to_save: 100
+
+# saved caches
+saved_caches_directory: target/embeddedCassandra/saved_caches
+
+# commitlog_sync may be either "periodic" or "batch."
+# When in batch mode, Cassandra won't ack writes until the commit log
+# has been fsynced to disk.  It will wait up to
+# commitlog_sync_batch_window_in_ms milliseconds for other writes, before
+# performing the sync.
+#
+# commitlog_sync: batch
+# commitlog_sync_batch_window_in_ms: 50
+#
+# the other option is "periodic" where writes may be acked immediately
+# and the CommitLog is simply synced every commitlog_sync_period_in_ms
+# milliseconds.
+commitlog_sync: periodic
+commitlog_sync_period_in_ms: 10000
+
+# The size of the individual commitlog file segments.  A commitlog
+# segment may be archived, deleted, or recycled once all the data
+# in it (potentially from each columnfamily in the system) has been
+# flushed to sstables.
+#
+# The default size is 32, which is almost always fine, but if you are
+# archiving commitlog segments (see commitlog_archiving.properties),
+# then you probably want a finer granularity of archiving; 8 or 16 MB
+# is reasonable.
+commitlog_segment_size_in_mb: 32
+
+# any class that implements the SeedProvider interface and has a
+# constructor that takes a Map<String, String> of parameters will do.
+seed_provider:
+    # Addresses of hosts that are deemed contact points.
+    # Cassandra nodes use this list of hosts to find each other and learn
+    # the topology of the ring.  You must change this if you are running
+    # multiple nodes!
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          # seeds is actually a comma-delimited list of addresses.
+          # Ex: "<ip1>,<ip2>,<ip3>"
+          - seeds: "127.0.0.1"
+
+
+# For workloads with more data than can fit in memory, Cassandra's
+# bottleneck will be reads that need to fetch data from
+# disk. "concurrent_reads" should be set to (16 * number_of_drives) in
+# order to allow the operations to enqueue low enough in the stack
+# that the OS and drives can reorder them.
+#
+# On the other hand, since writes are almost never IO bound, the ideal
+# number of "concurrent_writes" is dependent on the number of cores in
+# your system; (8 * number_of_cores) is a good rule of thumb.
+concurrent_reads: 32
+concurrent_writes: 32
+
+# Total memory to use for memtables.  Cassandra will flush the largest
+# memtable when this much memory is used.
+# If omitted, Cassandra will set it to 1/3 of the heap.
+# memtable_total_space_in_mb: 2048
+
+# Total space to use for commitlogs.
+# If space gets above this value (it will round up to the next nearest
+# segment multiple), Cassandra will flush every dirty CF in the oldest
+# segment and remove it.
+# commitlog_total_space_in_mb: 4096
+
+# This sets the amount of memtable flush writer threads.  These will
+# be blocked by disk io, and each one will hold a memtable in memory
+# while blocked. If you have a large heap and many data directories,
+# you can increase this value for better flush performance.
+# By default this will be set to the amount of data directories defined.
+#memtable_flush_writers: 1
+
+# the number of full memtables to allow pending flush, that is,
+# waiting for a writer thread.  At a minimum, this should be set to
+# the maximum number of secondary indexes created on a single CF.
+#memtable_flush_queue_size: 4
+
+# Whether to, when doing sequential writing, fsync() at intervals in
+# order to force the operating system to flush the dirty
+# buffers. Enable this to avoid sudden dirty buffer flushing from
+# impacting read latencies. Almost always a good idea on SSD:s; not
+# necessarily on platters.
+trickle_fsync: false
+trickle_fsync_interval_in_kb: 10240
+
+# TCP port, for commands and data
+storage_port: 7010
+
+# SSL port, for encrypted communication.  Unused unless enabled in
+# encryption_options
+ssl_storage_port: 7011
+
+# Address to bind to and tell other Cassandra nodes to connect to. You
+# _must_ change this if you want multiple nodes to be able to
+# communicate!
+#
+# Leaving it blank leaves it up to InetAddress.getLocalHost(). This
+# will always do the Right Thing *if* the node is properly configured
+# (hostname, name resolution, etc), and the Right Thing is to use the
+# address associated with the hostname (it might not be).
+#
+# Setting this to 0.0.0.0 is always wrong.
+listen_address: 127.0.0.1
+
+start_native_transport: true
+# port for the CQL native transport to listen for clients on
+native_transport_port: 9142
+
+# Whether to start the thrift rpc server.
+start_rpc: true
+
+# Address to broadcast to other Cassandra nodes
+# Leaving this blank will set it to the same value as listen_address
+# broadcast_address: 1.2.3.4
+
+# The address to bind the Thrift RPC service to -- clients connect
+# here. Unlike ListenAddress above, you *can* specify 0.0.0.0 here if
+# you want Thrift to listen on all interfaces.
+#
+# Leaving this blank has the same effect it does for ListenAddress,
+# (i.e. it will be based on the configured hostname of the node).
+rpc_address: localhost
+# port for Thrift to listen for clients on
+rpc_port: 9171
+
+# enable or disable keepalive on rpc connections
+rpc_keepalive: true
+
+# Cassandra provides three options for the RPC Server:
+#
+# sync  -> One connection per thread in the rpc pool (see below).
+#          For a very large number of clients, memory will be your limiting
+#          factor; on a 64 bit JVM, 128KB is the minimum stack size per thread.
+#          Connection pooling is very, very strongly recommended.
+#
+# async -> Nonblocking server implementation with one thread to serve
+#          rpc connections.  This is not recommended for high throughput use
+#          cases. Async has been tested to be about 50% slower than sync
+#          or hsha and is deprecated: it will be removed in the next major release.
+#
+# hsha  -> Stands for "half synchronous, half asynchronous." The rpc thread pool
+#          (see below) is used to manage requests, but the threads are multiplexed
+#          across the different clients.
+#
+# The default is sync because on Windows hsha is about 30% slower.  On Linux,
+# sync/hsha performance is about the same, with hsha of course using less memory.
+rpc_server_type: sync
+
+# Uncomment rpc_min|max|thread to set request pool size.
+# You would primarily set max for the sync server to safeguard against
+# misbehaved clients; if you do hit the max, Cassandra will block until one
+# disconnects before accepting more.  The defaults for sync are min of 16 and max
+# unlimited.
+#
+# For the Hsha server, the min and max both default to quadruple the number of
+# CPU cores.
+#
+# This configuration is ignored by the async server.
+#
+# rpc_min_threads: 16
+# rpc_max_threads: 2048
+
+# uncomment to set socket buffer sizes on rpc connections
+# rpc_send_buff_size_in_bytes:
+# rpc_recv_buff_size_in_bytes:
+
+# Frame size for thrift (maximum field length).
+# 0 disables TFramedTransport in favor of TSocket. This option
+# is deprecated; we strongly recommend using Framed mode.
+thrift_framed_transport_size_in_mb: 15
+
+# The max length of a thrift message, including all fields and
+# internal thrift overhead.
+thrift_max_message_length_in_mb: 16
+
+# Set to true to have Cassandra create a hard link to each sstable
+# flushed or streamed locally in a backups/ subdirectory of the
+# Keyspace data.  Removing these links is the operator's
+# responsibility.
+incremental_backups: false
+
+# Whether or not to take a snapshot before each compaction.  Be
+# careful using this option, since Cassandra won't clean up the
+# snapshots for you.  Mostly useful if you're paranoid when there
+# is a data format change.
+snapshot_before_compaction: false
+
+# Whether or not a snapshot is taken of the data before keyspace truncation
+# or dropping of column families. The STRONGLY advised default of true
+# should be used to provide data safety. If you set this flag to false, you will
+# lose data on truncation or drop.
+auto_snapshot: false
+
+# Add column indexes to a row after its contents reach this size.
+# Increase if your column values are large, or if you have a very large
+# number of columns.  The competing causes are, Cassandra has to
+# deserialize this much of the row to read a single column, so you want
+# it to be small - at least if you do many partial-row reads - but all
+# the index data is read for each access, so you don't want to generate
+# that wastefully either.
+column_index_size_in_kb: 64
+
+# Size limit for rows being compacted in memory.  Larger rows will spill
+# over to disk and use a slower two-pass compaction process.  A message
+# will be logged specifying the row key.
+#in_memory_compaction_limit_in_mb: 64
+
+# Number of simultaneous compactions to allow, NOT including
+# validation "compactions" for anti-entropy repair.  Simultaneous
+# compactions can help preserve read performance in a mixed read/write
+# workload, by mitigating the tendency of small sstables to accumulate
+# during a single long running compactions. The default is usually
+# fine and if you experience problems with compaction running too
+# slowly or too fast, you should look at
+# compaction_throughput_mb_per_sec first.
+#
+# This setting has no effect on LeveledCompactionStrategy.
+#
+# concurrent_compactors defaults to the number of cores.
+# Uncomment to make compaction mono-threaded, the pre-0.8 default.
+#concurrent_compactors: 1
+
+# Multi-threaded compaction. When enabled, each compaction will use
+# up to one thread per core, plus one thread per sstable being merged.
+# This is usually only useful for SSD-based hardware: otherwise,
+# your concern is usually to get compaction to do LESS i/o (see:
+# compaction_throughput_mb_per_sec), not more.
+#multithreaded_compaction: false
+
+# Throttles compaction to the given total throughput across the entire
+# system. The faster you insert data, the faster you need to compact in
+# order to keep the sstable count down, but in general, setting this to
+# 16 to 32 times the rate you are inserting data is more than sufficient.
+# Setting this to 0 disables throttling. Note that this account for all types
+# of compaction, including validation compaction.
+compaction_throughput_mb_per_sec: 16
+
+# Track cached row keys during compaction, and re-cache their new
+# positions in the compacted sstable.  Disable if you use really large
+# key caches.
+#compaction_preheat_key_cache: true
+
+# Throttles all outbound streaming file transfers on this node to the
+# given total throughput in Mbps. This is necessary because Cassandra does
+# mostly sequential IO when streaming data during bootstrap or repair, which
+# can lead to saturating the network connection and degrading rpc performance.
+# When unset, the default is 200 Mbps or 25 MB/s.
+# stream_throughput_outbound_megabits_per_sec: 200
+
+# How long the coordinator should wait for read operations to complete
+read_request_timeout_in_ms: 5000
+# How long the coordinator should wait for seq or index scans to complete
+range_request_timeout_in_ms: 10000
+# How long the coordinator should wait for writes to complete
+write_request_timeout_in_ms: 2000
+# How long a coordinator should continue to retry a CAS operation
+# that contends with other proposals for the same row
+cas_contention_timeout_in_ms: 1000
+# How long the coordinator should wait for truncates to complete
+# (This can be much longer, because unless auto_snapshot is disabled
+# we need to flush first so we can snapshot before removing the data.)
+truncate_request_timeout_in_ms: 60000
+# The default timeout for other, miscellaneous operations
+request_timeout_in_ms: 10000
+
+# Enable operation timeout information exchange between nodes to accurately
+# measure request timeouts.  If disabled, replicas will assume that requests
+# were forwarded to them instantly by the coordinator, which means that
+# under overload conditions we will waste that much extra time processing
+# already-timed-out requests.
+#
+# Warning: before enabling this property make sure to ntp is installed
+# and the times are synchronized between the nodes.
+cross_node_timeout: false
+
+# Enable socket timeout for streaming operation.
+# When a timeout occurs during streaming, streaming is retried from the start
+# of the current file. This _can_ involve re-streaming an important amount of
+# data, so you should avoid setting the value too low.
+# Default value is 0, which never timeout streams.
+# streaming_socket_timeout_in_ms: 0
+
+# phi value that must be reached for a host to be marked down.
+# most users should never need to adjust this.
+# phi_convict_threshold: 8
+
+# endpoint_snitch -- Set this to a class that implements
+# IEndpointSnitch.  The snitch has two functions:
+# - it teaches Cassandra enough about your network topology to route
+#   requests efficiently
+# - it allows Cassandra to spread replicas around your cluster to avoid
+#   correlated failures. It does this by grouping machines into
+#   "datacenters" and "racks."  Cassandra will do its best not to have
+#   more than one replica on the same "rack" (which may not actually
+#   be a physical location)
+#
+# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER,
+# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS
+# ARE PLACED.
+#
+# Out of the box, Cassandra provides
+#  - SimpleSnitch:
+#    Treats Strategy order as proximity. This improves cache locality
+#    when disabling read repair, which can further improve throughput.
+#    Only appropriate for single-datacenter deployments.
+#  - PropertyFileSnitch:
+#    Proximity is determined by rack and data center, which are
+#    explicitly configured in cassandra-topology.properties.
+#  - RackInferringSnitch:
+#    Proximity is determined by rack and data center, which are
+#    assumed to correspond to the 3rd and 2nd octet of each node's
+#    IP address, respectively.  Unless this happens to match your
+#    deployment conventions (as it did Facebook's), this is best used
+#    as an example of writing a custom Snitch class.
+#  - Ec2Snitch:
+#    Appropriate for EC2 deployments in a single Region.  Loads Region
+#    and Availability Zone information from the EC2 API. The Region is
+#    treated as the Datacenter, and the Availability Zone as the rack.
+#    Only private IPs are used, so this will not work across multiple
+#    Regions.
+#  - Ec2MultiRegionSnitch:
+#    Uses public IPs as broadcast_address to allow cross-region
+#    connectivity.  (Thus, you should set seed addresses to the public
+#    IP as well.) You will need to open the storage_port or
+#    ssl_storage_port on the public IP firewall.  (For intra-Region
+#    traffic, Cassandra will switch to the private IP after
+#    establishing a connection.)
+#
+# You can use a custom Snitch by setting this to the full class name
+# of the snitch, which will be assumed to be on your classpath.
+endpoint_snitch: SimpleSnitch
+
+# controls how often to perform the more expensive part of host score
+# calculation
+dynamic_snitch_update_interval_in_ms: 100
+# controls how often to reset all host scores, allowing a bad host to
+# possibly recover
+dynamic_snitch_reset_interval_in_ms: 600000
+# if set greater than zero and read_repair_chance is < 1.0, this will allow
+# 'pinning' of replicas to hosts in order to increase cache capacity.
+# The badness threshold will control how much worse the pinned host has to be
+# before the dynamic snitch will prefer other replicas over it.  This is
+# expressed as a double which represents a percentage.  Thus, a value of
+# 0.2 means Cassandra would continue to prefer the static snitch values
+# until the pinned host was 20% worse than the fastest.
+dynamic_snitch_badness_threshold: 0.1
+
+# request_scheduler -- Set this to a class that implements
+# RequestScheduler, which will schedule incoming client requests
+# according to the specific policy. This is useful for multi-tenancy
+# with a single Cassandra cluster.
+# NOTE: This is specifically for requests from the client and does
+# not affect inter node communication.
+# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place
+# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of
+# client requests to a node with a separate queue for each
+# request_scheduler_id. The scheduler is further customized by
+# request_scheduler_options as described below.
+request_scheduler: org.apache.cassandra.scheduler.NoScheduler
+
+# Scheduler Options vary based on the type of scheduler
+# NoScheduler - Has no options
+# RoundRobin
+#  - throttle_limit -- The throttle_limit is the number of in-flight
+#                      requests per client.  Requests beyond
+#                      that limit are queued up until
+#                      running requests can complete.
+#                      The value of 80 here is twice the number of
+#                      concurrent_reads + concurrent_writes.
+#  - default_weight -- default_weight is optional and allows for
+#                      overriding the default which is 1.
+#  - weights -- Weights are optional and will default to 1 or the
+#               overridden default_weight. The weight translates into how
+#               many requests are handled during each turn of the
+#               RoundRobin, based on the scheduler id.
+#
+# request_scheduler_options:
+#    throttle_limit: 80
+#    default_weight: 5
+#    weights:
+#      Keyspace1: 1
+#      Keyspace2: 5
+
+# request_scheduler_id -- An identifer based on which to perform
+# the request scheduling. Currently the only valid option is keyspace.
+# request_scheduler_id: keyspace
+
+# index_interval controls the sampling of entries from the primrary
+# row index in terms of space versus time.  The larger the interval,
+# the smaller and less effective the sampling will be.  In technicial
+# terms, the interval coresponds to the number of index entries that
+# are skipped between taking each sample.  All the sampled entries
+# must fit in memory.  Generally, a value between 128 and 512 here
+# coupled with a large key cache size on CFs results in the best trade
+# offs.  This value is not often changed, however if you have many
+# very small rows (many to an OS page), then increasing this will
+# often lower memory usage without a impact on performance.
+index_interval: 128
+
+# Enable or disable inter-node encryption
+# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that
+# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher
+# suite for authentication, key exchange and encryption of the actual data transfers.
+# NOTE: No custom encryption options are enabled at the moment
+# The available internode options are : all, none, dc, rack
+#
+# If set to dc cassandra will encrypt the traffic between the DCs
+# If set to rack cassandra will encrypt the traffic between the racks
+#
+# The passwords used in these options must match the passwords used when generating
+# the keystore and truststore.  For instructions on generating these files, see:
+# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore
+#
+encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+    # More advanced defaults below:
+    # protocol: TLS
+    # algorithm: SunX509
+    # store_type: JKS
+    # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/cassandra/src/test/resources/logback-test.xml b/cassandra/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..edfc5b0
--- /dev/null
+++ b/cassandra/src/test/resources/logback-test.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!-- Cassandra Unit uses logback API for logging -->
+<configuration>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%d{yyyy-MM-dd HH:mm:ss} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <root level="error">
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/calcite/blob/bd0e1400/cassandra/src/test/resources/model.json
----------------------------------------------------------------------
diff --git a/cassandra/src/test/resources/model.json b/cassandra/src/test/resources/model.json
index 5713d11..ea9c5d5 100644
--- a/cassandra/src/test/resources/model.json
+++ b/cassandra/src/test/resources/model.json
@@ -24,6 +24,7 @@
       "factory": "org.apache.calcite.adapter.cassandra.CassandraSchemaFactory",
       "operand": {
         "host": "localhost",
+        "port": 9142,
         "keyspace": "twissandra"
       }
     }