You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/09/06 01:56:41 UTC

[1/4] git commit: Enable connection pooling in test suite.

Repository: incubator-optiq
Updated Branches:
  refs/heads/master d63f5d561 -> 6fbefcec9


Enable connection pooling in test suite.

Add FoodmartTest to suite (enabled if -Doptiq.test.slow=true).

Switch various callbacks in the test suite from linq4j Function1 to guava Function.

Add test case for [OPTIQ-403].


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

Branch: refs/heads/master
Commit: d62c064f770f38af4e113f8beb95cf4d32373474
Parents: d63f5d5
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Sep 5 12:08:02 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 5 12:08:02 2014 -0700

----------------------------------------------------------------------
 .../optiq/impl/jdbc/JdbcImplementor.java        |   1 +
 .../hydromatic/optiq/impl/jdbc/JdbcTable.java   |  40 +++--
 .../net/hydromatic/optiq/test/FoodmartTest.java |  18 +-
 .../optiq/test/JdbcFrontJdbcBackTest.java       |  12 +-
 .../net/hydromatic/optiq/test/JdbcTest.java     | 138 ++++++++-------
 .../net/hydromatic/optiq/test/LatticeTest.java  |   4 +-
 .../optiq/test/MaterializationTest.java         |   8 +-
 .../net/hydromatic/optiq/test/OptiqAssert.java  | 170 ++++++++++++++-----
 .../net/hydromatic/optiq/test/OptiqSuite.java   |   6 +-
 .../optiq/test/ReflectiveSchemaTest.java        |   4 +-
 .../hydromatic/optiq/test/MongoAdapterTest.java |  10 +-
 .../hydromatic/optiq/impl/tpch/TpchTest.java    |  18 +-
 12 files changed, 285 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcImplementor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcImplementor.java b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcImplementor.java
index c4340ae..104fba2 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcImplementor.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcImplementor.java
@@ -143,6 +143,7 @@ public class JdbcImplementor {
           return SqlLiteral.createTimestamp((Calendar) literal.getValue(),
               literal.getType().getPrecision(), POS);
         case ANY:
+        case NULL:
           switch (literal.getTypeName()) {
           case NULL:
             return SqlLiteral.createNull(POS);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcTable.java b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcTable.java
index 2b137ed..a3368ad 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcTable.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcTable.java
@@ -144,20 +144,32 @@ class JdbcTable extends AbstractQueryableTable implements TranslatableTable {
 
   public <T> Queryable<T> asQueryable(QueryProvider queryProvider,
       SchemaPlus schema, String tableName) {
-    return new AbstractTableQueryable<T>(queryProvider, schema, this,
-        tableName) {
-      public Enumerator<T> enumerator() {
-        final JavaTypeFactory typeFactory =
-            ((OptiqConnection) queryProvider).getTypeFactory();
-        final SqlString sql = generateSql();
-        //noinspection unchecked
-        final Enumerable<T> enumerable = (Enumerable<T>) ResultSetEnumerable.of(
-            jdbcSchema.getDataSource(),
-            sql.getSql(),
-            JdbcUtils.ObjectArrayRowBuilder.factory(fieldClasses(typeFactory)));
-        return enumerable.enumerator();
-      }
-    };
+    return new JdbcTableQueryable<T>(queryProvider, schema, tableName);
+  }
+
+  /** Enumerable that returns the contents of a {@link JdbcTable} by connecting
+   * to the JDBC data source. */
+  private class JdbcTableQueryable<T> extends AbstractTableQueryable<T> {
+    public JdbcTableQueryable(QueryProvider queryProvider, SchemaPlus schema,
+        String tableName) {
+      super(queryProvider, schema, JdbcTable.this, tableName);
+    }
+
+    @Override public String toString() {
+      return "JdbcTableQueryable {table: " + tableName + "}";
+    }
+
+    public Enumerator<T> enumerator() {
+      final JavaTypeFactory typeFactory =
+          ((OptiqConnection) queryProvider).getTypeFactory();
+      final SqlString sql = generateSql();
+      //noinspection unchecked
+      final Enumerable<T> enumerable = (Enumerable<T>) ResultSetEnumerable.of(
+          jdbcSchema.getDataSource(),
+          sql.getSql(),
+          JdbcUtils.ObjectArrayRowBuilder.factory(fieldClasses(typeFactory)));
+      return enumerable.enumerator();
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java b/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
index 06ea094..f4669e3 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
@@ -16,6 +16,8 @@
  */
 package net.hydromatic.optiq.test;
 
+import net.hydromatic.linq4j.expressions.Primitive;
+
 import org.eigenbase.util.IntegerIntervalSet;
 
 import com.fasterxml.jackson.core.JsonParser;
@@ -78,6 +80,13 @@ public class FoodmartTest {
     6564, 6566, 6578, 6581, 6582, 6583, 6587, 6591, 6594, 6603, 6610, 6613,
     6615, 6618, 6619, 6622, 6627, 6632, 6635, 6643, 6650, 6651, 6652, 6653,
     6656, 6659, 6668, 6670, 6720, 6726, 6735, 6737, 6739,
+
+    // timeout oor OOM
+    420, 423, 5218, 5219, 5616, 5617, 5618, 5891, 5892, 5895, 5896, 5898, 5899,
+    5900, 5901, 5902, 6080, 6091,
+
+    // bugs
+    6597, // OPTIQ-403
   };
 
   // Interesting tests. (We need to fix and remove from the disabled list.)
@@ -101,7 +110,6 @@ public class FoodmartTest {
       for (int disabledId : DISABLED_IDS) {
         buf.append(",-").append(disabledId);
       }
-      buf.setLength(0); // disable disable
       for (Integer id : IntegerIntervalSet.of(idList + buf)) {
         final FoodmartQuery query1 = set.queries.get(id);
         if (query1 != null) {
@@ -110,6 +118,13 @@ public class FoodmartTest {
       }
     } else {
       for (FoodmartQuery query1 : set.queries.values()) {
+        if (!OptiqAssert.ENABLE_SLOW && query1.id != 2) {
+          // If slow queries are not enabled, only run query #2.
+          continue;
+        }
+        if (Primitive.asList(DISABLED_IDS).contains(query1.id)) {
+          continue;
+        }
         list.add(new Object[]{query1.id /*, query1.sql */});
       }
     }
@@ -127,6 +142,7 @@ public class FoodmartTest {
       OptiqAssert.that()
 //          .withModel(JdbcTest.FOODMART_MODEL)
           .with(OptiqAssert.Config.FOODMART_CLONE)
+          .pooled()
 //          .withSchema("foodmart")
           .query(query.sql)
           .runs();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/JdbcFrontJdbcBackTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/JdbcFrontJdbcBackTest.java b/core/src/test/java/net/hydromatic/optiq/test/JdbcFrontJdbcBackTest.java
index a367a10..b8fb934 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/JdbcFrontJdbcBackTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/JdbcFrontJdbcBackTest.java
@@ -16,10 +16,10 @@
  */
 package net.hydromatic.optiq.test;
 
-import net.hydromatic.linq4j.function.Function1;
-
 import net.hydromatic.optiq.jdbc.OptiqConnection;
 
+import com.google.common.base.Function;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -54,7 +54,7 @@ public class JdbcFrontJdbcBackTest {
     that()
         .with(OptiqAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   ResultSet rset =
@@ -79,7 +79,7 @@ public class JdbcFrontJdbcBackTest {
     that()
         .with(OptiqAssert.Config.REGULAR_PLUS_METADATA)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   ResultSet rset =
@@ -105,7 +105,7 @@ public class JdbcFrontJdbcBackTest {
     that()
         .with(OptiqAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   ResultSet rset =
@@ -133,7 +133,7 @@ public class JdbcFrontJdbcBackTest {
     that()
         .with(OptiqAssert.Config.JDBC_FOODMART)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   ResultSet rset =

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
index 471f973..2efcbaa 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
@@ -899,7 +899,7 @@ public class JdbcTest {
     // the first instance of the column will be returned."
     OptiqAssert.that()
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection c) {
                 try {
                   Statement s = c.createStatement();
@@ -1440,7 +1440,7 @@ public class JdbcTest {
         .query(
             "values extract(year from date '2008-2-23')")
         .returns(
-            new Function1<ResultSet, Void>() {
+            new Function<ResultSet, Void>() {
               public Void apply(ResultSet a0) {
                 try {
                   final BigDecimal bigDecimal = a0.getBigDecimal(1);
@@ -1797,7 +1797,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   final Statement statement = a0.createStatement();
@@ -2986,6 +2986,27 @@ public class JdbcTest {
         .returnsCount(0);
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/OPTIQ-403">OPTIQ-403</a>,
+   * "Enumerable gives NullPointerException with HAVING on nullable
+   * expression". */
+  @Ignore("OPTIQ-403")
+  @Test public void testHavingNot() throws IOException {
+    withFoodMartQuery(6597).runs();
+  }
+
+  /** Minimal case of {@link #testHavingNot()}. */
+  @Ignore("OPTIQ-403")
+  @Test public void testHavingNot2() throws IOException {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.FOODMART_CLONE)
+        .query("select 1\n"
+            + "from \"store\"\n"
+            + "group by \"store\".\"store_street_address\"\n"
+            + "having NOT (sum(\"store\".\"grocery_sqft\") < 10000)")
+        .returnsCount(0);
+  }
+
   /** Query that reads no columns from either underlying table. */
   @Test public void testCountStar() {
     OptiqAssert.that()
@@ -4185,13 +4206,14 @@ public class JdbcTest {
             + "  from \"hr\".\"depts\" as d2\n"
             + "  join \"hr\".\"emps\" as e2 using (\"deptno\")\n"
             + "where d.\"deptno\" = d2.\"deptno\")")
-        .convertMatches(new Function1<RelNode, Void>() {
-          public Void apply(RelNode relNode) {
-            String s = RelOptUtil.toString(relNode);
-            assertThat(s, not(containsString("CorrelatorRel")));
-            return null;
-          }
-        });
+        .convertMatches(
+            new Function<RelNode, Void>() {
+              public Void apply(RelNode relNode) {
+                String s = RelOptUtil.toString(relNode);
+                assertThat(s, not(containsString("CorrelatorRel")));
+                return null;
+              }
+            });
   }
 
   /** Tests a correlated scalar sub-query in the SELECT clause.
@@ -4467,7 +4489,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   final Statement statement = a0.createStatement();
@@ -4499,7 +4521,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(OptiqAssert.Config.REGULAR)
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection connection) {
                 try {
                   final PreparedStatement preparedStatement =
@@ -4670,7 +4692,7 @@ public class JdbcTest {
             + "}");
     // check that the specified 'defaultSchema' was used
     that.doWithConnection(
-        new Function1<OptiqConnection, Object>() {
+        new Function<OptiqConnection, Object>() {
           public Object apply(OptiqConnection connection) {
             try {
               assertEquals("adhoc", connection.getSchema());
@@ -4764,7 +4786,7 @@ public class JdbcTest {
 
     // Make sure that views appear in metadata.
     with.doWithConnection(
-        new Function1<OptiqConnection, Void>() {
+        new Function<OptiqConnection, Void>() {
           public Void apply(OptiqConnection a0) {
             try {
               final DatabaseMetaData metaData = a0.getMetaData();
@@ -4874,7 +4896,7 @@ public class JdbcTest {
               }
             })
         .doWithConnection(
-            new Function1<OptiqConnection, Object>() {
+            new Function<OptiqConnection, Object>() {
               public Object apply(OptiqConnection a0) {
                 try {
                   a0.createStatement()
@@ -5294,7 +5316,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("timezone", "GMT+1:00"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   final PreparedStatement statement =
@@ -5333,7 +5355,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("timezone", "GMT+1:00"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   checkGetTimestamp(connection);
@@ -5480,23 +5502,24 @@ public class JdbcTest {
   public void testGetDate() throws Exception {
     OptiqAssert.that()
       .with(OptiqAssert.Config.JDBC_FOODMART)
-      .doWithConnection(new Function1<OptiqConnection, Object>() {
-        public Object apply(OptiqConnection conn) {
-          try {
-            Statement stmt = conn.createStatement();
-            ResultSet rs = stmt.executeQuery(
-              "select min(\"date\") mindate from \"foodmart\".\"currency\"");
-            assertTrue(rs.next());
-            assertEquals(
-              Date.valueOf("1997-01-01"),
-              rs.getDate(1));
-            assertFalse(rs.next());
-            return null;
-          } catch (SQLException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      });
+      .doWithConnection(
+          new Function<OptiqConnection, Object>() {
+            public Object apply(OptiqConnection conn) {
+              try {
+                Statement stmt = conn.createStatement();
+                ResultSet rs = stmt.executeQuery(
+                    "select min(\"date\") mindate from \"foodmart\".\"currency\"");
+                assertTrue(rs.next());
+                assertEquals(
+                    Date.valueOf("1997-01-01"),
+                    rs.getDate(1));
+                assertFalse(rs.next());
+                return null;
+              } catch (SQLException e) {
+                throw new RuntimeException(e);
+              }
+            }
+          });
   }
 
   /** Tests accessing a date as a string in a JDBC source whose type is DATE. */
@@ -5512,23 +5535,24 @@ public class JdbcTest {
   public void testGetTimestampObject() throws Exception {
     OptiqAssert.that()
       .with(OptiqAssert.Config.JDBC_FOODMART)
-      .doWithConnection(new Function1<OptiqConnection, Object>() {
-        public Object apply(OptiqConnection conn) {
-          try {
-            Statement stmt = conn.createStatement();
-            ResultSet rs = stmt.executeQuery(
-              "select \"hire_date\" from \"foodmart\".\"employee\" where \"employee_id\" = 1");
-            assertTrue(rs.next());
-            assertEquals(
-              Timestamp.valueOf("1994-12-01 00:00:00"),
-              rs.getTimestamp(1));
-            assertFalse(rs.next());
-            return null;
-          } catch (SQLException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      });
+      .doWithConnection(
+          new Function<OptiqConnection, Object>() {
+            public Object apply(OptiqConnection conn) {
+              try {
+                Statement stmt = conn.createStatement();
+                ResultSet rs = stmt.executeQuery(
+                    "select \"hire_date\" from \"foodmart\".\"employee\" where \"employee_id\" = 1");
+                assertTrue(rs.next());
+                assertEquals(
+                    Timestamp.valueOf("1994-12-01 00:00:00"),
+                    rs.getTimestamp(1));
+                assertFalse(rs.next());
+                return null;
+              } catch (SQLException e) {
+                throw new RuntimeException(e);
+              }
+            }
+          });
   }
 
   @Test public void testUnicode() throws Exception {
@@ -5571,7 +5595,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("lex", "MYSQL"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   DatabaseMetaData metaData = connection.getMetaData();
@@ -5605,7 +5629,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("lex", "SQL_SERVER"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   DatabaseMetaData metaData = connection.getMetaData();
@@ -5639,7 +5663,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("lex", "ORACLE"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   DatabaseMetaData metaData = connection.getMetaData();
@@ -5677,7 +5701,7 @@ public class JdbcTest {
     OptiqAssert.that()
         .with(ImmutableMap.of("lex", "JAVA"))
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   DatabaseMetaData metaData = connection.getMetaData();
@@ -5707,7 +5731,7 @@ public class JdbcTest {
             });
   }
 
-  /** Tests metadata for the ORACLE lexical scheme overriden like JAVA. */
+  /** Tests metadata for the ORACLE lexical scheme overridden like JAVA. */
   @Test public void testLexOracleAsJava() throws Exception {
     OptiqAssert.that()
         .with(ImmutableMap.<String, String>builder()
@@ -5718,7 +5742,7 @@ public class JdbcTest {
             .put("caseSensitive", "TRUE")
             .build())
         .doWithConnection(
-            new Function1<OptiqConnection, Void>() {
+            new Function<OptiqConnection, Void>() {
               public Void apply(OptiqConnection connection) {
                 try {
                   DatabaseMetaData metaData = connection.getMetaData();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java b/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
index e1cacd2..c09d444 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/LatticeTest.java
@@ -16,8 +16,6 @@
  */
 package net.hydromatic.optiq.test;
 
-import net.hydromatic.linq4j.function.Function1;
-
 import net.hydromatic.optiq.runtime.Hook;
 
 import org.eigenbase.rel.RelNode;
@@ -193,7 +191,7 @@ public class LatticeTest {
             + "join \"foodmart\".\"product\" as p using (\"product_id\")\n")
         .enableMaterializations(true)
         .substitutionMatches(
-            new Function1<RelNode, Void>() {
+            new Function<RelNode, Void>() {
               public Void apply(RelNode relNode) {
                 counter.incrementAndGet();
                 String s = RelOptUtil.toString(relNode);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java b/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
index 76aaa92..314a919 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/MaterializationTest.java
@@ -16,8 +16,6 @@
  */
 package net.hydromatic.optiq.test;
 
-import net.hydromatic.linq4j.function.Function1;
-
 import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
 import net.hydromatic.optiq.materialize.MaterializationService;
 import net.hydromatic.optiq.prepare.Prepare;
@@ -27,6 +25,8 @@ import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.rex.*;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 
+import com.google.common.base.Function;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -42,7 +42,7 @@ import static org.junit.Assert.*;
  * and checks that the materialization is used.
  */
 public class MaterializationTest {
-  private static final Function1<ResultSet, Void> CONTAINS_M0 =
+  private static final Function<ResultSet, Void> CONTAINS_M0 =
       OptiqAssert.checkResultContains(
           "EnumerableTableAccessRel(table=[[hr, m0]])");
 
@@ -94,7 +94,7 @@ public class MaterializationTest {
   /** Checks that a given query can use a materialized view with a given
    * definition. */
   private void checkMaterialize(String materialize, String query, String model,
-      Function1<ResultSet, Void> explainChecker) {
+      Function<ResultSet, Void> explainChecker) {
     try {
       Prepare.THREAD_TRIM.set(true);
       MaterializationService.setThreadLocal();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java b/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
index 4cc87ce..c2967f2 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
@@ -16,8 +16,6 @@
  */
 package net.hydromatic.optiq.test;
 
-import net.hydromatic.linq4j.function.Function1;
-
 import net.hydromatic.optiq.*;
 import net.hydromatic.optiq.config.OptiqConnectionProperty;
 import net.hydromatic.optiq.impl.AbstractSchema;
@@ -33,7 +31,9 @@ import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
 import org.eigenbase.util.*;
 
+import com.google.common.base.*;
 import com.google.common.base.Function;
+import com.google.common.cache.*;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultiset;
 import com.google.common.collect.Lists;
@@ -122,12 +122,12 @@ public class OptiqAssert {
         }
 
         @Override public AssertThat connectThrows(
-            Function1<Throwable, Void> exceptionChecker) {
+            Function<Throwable, Void> exceptionChecker) {
           return this;
         }
 
         @Override
-        public <T> AssertThat doWithConnection(Function1<OptiqConnection, T> fn)
+        public <T> AssertThat doWithConnection(Function<OptiqConnection, T> fn)
             throws Exception {
           return this;
         }
@@ -141,6 +141,11 @@ public class OptiqAssert {
         public AssertThat enable(boolean enabled) {
           return this;
         }
+
+        @Override
+        public AssertThat pooled() {
+          return this;
+        }
       };
 
   /** Creates an instance of {@code OptiqAssert} with the regular
@@ -149,9 +154,9 @@ public class OptiqAssert {
     return new AssertThat(Config.REGULAR);
   }
 
-  static Function1<RelNode, Void> checkRel(final String expected,
+  static Function<RelNode, Void> checkRel(final String expected,
       final AtomicInteger counter) {
-    return new Function1<RelNode, Void>() {
+    return new Function<RelNode, Void>() {
       public Void apply(RelNode relNode) {
         if (counter != null) {
           counter.incrementAndGet();
@@ -163,9 +168,9 @@ public class OptiqAssert {
     };
   }
 
-  static Function1<Throwable, Void> checkException(
+  static Function<Throwable, Void> checkException(
       final String expected) {
-    return new Function1<Throwable, Void>() {
+    return new Function<Throwable, Void>() {
       public Void apply(Throwable p0) {
         assertNotNull(
             "expected exception but none was thrown", p0);
@@ -180,8 +185,8 @@ public class OptiqAssert {
     };
   }
 
-  static Function1<ResultSet, Void> checkResult(final String expected) {
-    return new Function1<ResultSet, Void>() {
+  static Function<ResultSet, Void> checkResult(final String expected) {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
           final String resultString = OptiqAssert.toString(resultSet);
@@ -194,8 +199,8 @@ public class OptiqAssert {
     };
   }
 
-  static Function1<ResultSet, Void> checkResultValue(final String expected) {
-    return new Function1<ResultSet, Void>() {
+  static Function<ResultSet, Void> checkResultValue(final String expected) {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
           if (!resultSet.next()) {
@@ -214,8 +219,8 @@ public class OptiqAssert {
     };
   }
 
-  static Function1<ResultSet, Void> checkResultCount(final int expected) {
-    return new Function1<ResultSet, Void>() {
+  static Function<ResultSet, Void> checkResultCount(final int expected) {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
           final int count = OptiqAssert.countRows(resultSet);
@@ -233,8 +238,8 @@ public class OptiqAssert {
    *
    * @param ordered Whether order should be the same both times
    */
-  static Function1<ResultSet, Void> consistentResult(final boolean ordered) {
-    return new Function1<ResultSet, Void>() {
+  static Function<ResultSet, Void> consistentResult(final boolean ordered) {
+    return new Function<ResultSet, Void>() {
       int executeCount = 0;
       Collection expected;
 
@@ -269,9 +274,9 @@ public class OptiqAssert {
     return buf.toString();
   }
 
-  static Function1<ResultSet, Void> checkResultUnordered(
+  static Function<ResultSet, Void> checkResultUnordered(
       final String... lines) {
-    return new Function1<ResultSet, Void>() {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
           final List<String> expectedList = Lists.newArrayList(lines);
@@ -291,9 +296,9 @@ public class OptiqAssert {
     };
   }
 
-  public static Function1<ResultSet, Void> checkResultContains(
+  public static Function<ResultSet, Void> checkResultContains(
       final String expected) {
-    return new Function1<ResultSet, Void>() {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet s) {
         try {
           final String actual = Util.toLinux(OptiqAssert.toString(s));
@@ -308,9 +313,9 @@ public class OptiqAssert {
     };
   }
 
-  public static Function1<ResultSet, Void> checkMaskedResultContains(
+  public static Function<ResultSet, Void> checkMaskedResultContains(
       final String expected) {
-    return new Function1<ResultSet, Void>() {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet s) {
         try {
           final String actual = Util.toLinux(OptiqAssert.toString(s));
@@ -327,9 +332,9 @@ public class OptiqAssert {
     };
   }
 
-  public static Function1<ResultSet, Void> checkResultType(
+  public static Function<ResultSet, Void> checkResultType(
       final String expected) {
-    return new Function1<ResultSet, Void>() {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet s) {
         try {
           final String actual = typeString(s.getMetaData());
@@ -363,8 +368,8 @@ public class OptiqAssert {
       int limit,
       boolean materializationsEnabled,
       List<Pair<Hook, Function>> hooks,
-      Function1<ResultSet, Void> resultChecker,
-      Function1<Throwable, Void> exceptionChecker) throws Exception {
+      Function<ResultSet, Void> resultChecker,
+      Function<Throwable, Void> exceptionChecker) throws Exception {
     final String message =
         "With materializationsEnabled=" + materializationsEnabled
         + ", limit=" + limit;
@@ -420,8 +425,8 @@ public class OptiqAssert {
       Connection connection,
       String sql,
       boolean materializationsEnabled,
-      final Function1<RelNode, Void> convertChecker,
-      final Function1<RelNode, Void> substitutionChecker) throws Exception {
+      final Function<RelNode, Void> convertChecker,
+      final Function<RelNode, Void> substitutionChecker) throws Exception {
     final String message =
         "With materializationsEnabled=" + materializationsEnabled;
     final Hook.Closeable closeable =
@@ -682,6 +687,11 @@ public class OptiqAssert {
     return optiqConnection;
   }
 
+  static <F, T> Function<F, T> constantNull() {
+    //noinspection unchecked
+    return (Function<F, T>) (Function) Functions.<T>constant(null);
+  }
+
   /**
    * Result of calling {@link OptiqAssert#that}.
    */
@@ -801,7 +811,7 @@ public class OptiqAssert {
     /** Asserts that there is an exception that matches the given predicate
      * while creating a connection. */
     public AssertThat connectThrows(
-        Function1<Throwable, Void> exceptionChecker) {
+        Function<Throwable, Void> exceptionChecker) {
       Throwable throwable;
       try {
         Connection x = connectionFactory.createConnection();
@@ -819,7 +829,7 @@ public class OptiqAssert {
     }
 
     /** Creates a {@link OptiqConnection} and executes a callback. */
-    public <T> AssertThat doWithConnection(Function1<OptiqConnection, T> fn)
+    public <T> AssertThat doWithConnection(Function<OptiqConnection, T> fn)
         throws Exception {
       Connection connection = connectionFactory.createConnection();
       try {
@@ -832,7 +842,7 @@ public class OptiqAssert {
     }
 
     /** Creates a {@link DataContext} and executes a callback. */
-    public <T> AssertThat doWithDataContext(Function1<DataContext, T> fn)
+    public <T> AssertThat doWithDataContext(Function<DataContext, T> fn)
         throws Exception {
       OptiqConnection connection = connectionFactory.createConnection();
       final DataContext dataContext = MetaImpl.createDataContext(connection);
@@ -858,12 +868,78 @@ public class OptiqAssert {
     public AssertThat enable(boolean enabled) {
       return enabled ? this : DISABLED;
     }
+
+    /** Returns a version that uses a single connection, as opposed to creating
+     * a new one each time a test method is invoked. */
+    public AssertThat pooled() {
+      if (connectionFactory instanceof PoolingConnectionFactory) {
+        return this;
+      } else {
+        return new AssertThat(new PoolingConnectionFactory(connectionFactory));
+      }
+    }
   }
 
   public interface ConnectionFactory {
     OptiqConnection createConnection() throws Exception;
   }
 
+  private static class MemoizingConnectionFactory implements ConnectionFactory {
+    private final Supplier<OptiqConnection> supplier;
+
+    public MemoizingConnectionFactory(final ConnectionFactory factory) {
+      super();
+      this.supplier = Suppliers.memoize(
+          new Supplier<OptiqConnection>() {
+            public OptiqConnection get() {
+              try {
+                return factory.createConnection();
+              } catch (Exception e) {
+                throw new RuntimeException(e);
+              }
+            }
+          });
+    }
+
+    public OptiqConnection createConnection() throws Exception {
+      try {
+        return supplier.get();
+      } catch (RuntimeException e) {
+        if (e.getClass() == RuntimeException.class
+            && e.getCause() instanceof Exception
+            && e.getCause() != e) {
+          throw (Exception) e.getCause();
+        }
+        throw e;
+      }
+    }
+  }
+
+  private static class PoolingConnectionFactory implements ConnectionFactory {
+    private final ConnectionFactory factory;
+
+    public PoolingConnectionFactory(final ConnectionFactory factory) {
+      this.factory = factory;
+    }
+
+    public OptiqConnection createConnection() throws Exception {
+      return Pool.INSTANCE.cache.get(factory);
+    }
+  }
+
+  private static class Pool {
+    private static final Pool INSTANCE = new Pool();
+
+    private final LoadingCache<ConnectionFactory, OptiqConnection> cache =
+        CacheBuilder.newBuilder().build(
+            new CacheLoader<ConnectionFactory, OptiqConnection>() {
+              public OptiqConnection load(ConnectionFactory key)
+                  throws Exception {
+                return key.createConnection();
+              }
+            });
+  }
+
   private static class ConfigConnectionFactory implements ConnectionFactory {
     private final Config config;
 
@@ -871,6 +947,16 @@ public class OptiqAssert {
       this.config = config;
     }
 
+    @Override public int hashCode() {
+      return config.hashCode();
+    }
+
+    @Override public boolean equals(Object obj) {
+      return obj == this
+          || obj instanceof ConfigConnectionFactory
+          && config == ((ConfigConnectionFactory) obj).config;
+    }
+
     public OptiqConnection createConnection() throws Exception {
       switch (config) {
       case REGULAR:
@@ -954,12 +1040,12 @@ public class OptiqAssert {
       return returns(checkResultCount(expectedCount));
     }
 
-    public final AssertQuery returns(Function1<ResultSet, Void> checker) {
+    public final AssertQuery returns(Function<ResultSet, Void> checker) {
       return returns(sql, checker);
     }
 
     protected AssertQuery returns(String sql,
-        Function1<ResultSet, Void> checker) {
+        Function<ResultSet, Void> checker) {
       try {
         assertQuery(createConnection(), sql, limit, materializationsEnabled,
             hooks, checker, null);
@@ -1014,7 +1100,7 @@ public class OptiqAssert {
       return convertMatches(checkRel(expected, null));
     }
 
-    public AssertQuery convertMatches(final Function1<RelNode, Void> checker) {
+    public AssertQuery convertMatches(final Function<RelNode, Void> checker) {
       try {
         assertPrepare(createConnection(), sql, this.materializationsEnabled,
             checker, null);
@@ -1026,7 +1112,7 @@ public class OptiqAssert {
     }
 
     public AssertQuery substitutionMatches(
-        final Function1<RelNode, Void> checker) {
+        final Function<RelNode, Void> checker) {
       try {
         assertPrepare(createConnection(), sql, materializationsEnabled,
             null, checker);
@@ -1042,7 +1128,7 @@ public class OptiqAssert {
     }
 
     public final AssertQuery explainMatches(String extra,
-        Function1<ResultSet, Void> checker) {
+        Function<ResultSet, Void> checker) {
       return returns("explain plan " + extra + "for " + sql, checker);
     }
 
@@ -1090,7 +1176,7 @@ public class OptiqAssert {
      * queries. The checker should throw to fail the test if it does not see
      * what it wants. This method can be used to check whether a particular
      * MongoDB or SQL query is generated, for instance. */
-    public AssertQuery queryContains(Function1<List, Void> predicate1) {
+    public AssertQuery queryContains(Function<List, Void> predicate1) {
       final List<Object> list = Lists.newArrayList();
       addHook(Hook.QUERY_PLAN,
           new Function<Object, Void>() {
@@ -1121,7 +1207,7 @@ public class OptiqAssert {
       try {
         materializationsEnabled = false;
         final boolean ordered = sql.toUpperCase().contains("ORDER BY");
-        final Function1<ResultSet, Void> checker = consistentResult(ordered);
+        final Function<ResultSet, Void> checker = consistentResult(ordered);
         returns(checker);
         materializationsEnabled = true;
         returns(checker);
@@ -1199,7 +1285,7 @@ public class OptiqAssert {
     }
 
     @Override
-    public AssertQuery returns(String sql, Function1<ResultSet, Void> checker) {
+    public AssertQuery returns(String sql, Function<ResultSet, Void> checker) {
       return this;
     }
 
@@ -1214,12 +1300,12 @@ public class OptiqAssert {
     }
 
     @Override public AssertQuery convertMatches(
-        Function1<RelNode, Void> checker) {
+        Function<RelNode, Void> checker) {
       return this;
     }
 
     @Override public AssertQuery substitutionMatches(
-        Function1<RelNode, Void> checker) {
+        Function<RelNode, Void> checker) {
       return this;
     }
 
@@ -1234,7 +1320,7 @@ public class OptiqAssert {
     }
 
     @Override
-    public AssertQuery queryContains(Function1<List, Void> predicate1) {
+    public AssertQuery queryContains(Function<List, Void> predicate1) {
       return this;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java b/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
index 1e58866..7c3702f 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/OptiqSuite.java
@@ -112,7 +112,11 @@ import org.junit.runners.Suite;
     // slow tests that don't break often
     SqlToRelConverterExtendedTest.class,
     SqlRunTest.class,
-    PartiallyOrderedSetTest.class
+    PartiallyOrderedSetTest.class,
+
+    // system tests and benchmarks (very slow, but usually only run if
+    // '-Doptiq.test.slow=true' is specified)
+    FoodmartTest.class
 })
 public class OptiqSuite {
 }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/core/src/test/java/net/hydromatic/optiq/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/ReflectiveSchemaTest.java b/core/src/test/java/net/hydromatic/optiq/test/ReflectiveSchemaTest.java
index 700d7cf..33a1380 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/ReflectiveSchemaTest.java
@@ -288,7 +288,7 @@ public class ReflectiveSchemaTest {
       with.query(
           "select " + fn + "(\"" + field.getName() + "\") as c\n"
           + "from \"s\".\"everyTypes\"")
-          .returns(Functions.<ResultSet, Void>constantNull());
+          .returns(OptiqAssert.<ResultSet, Void>constantNull());
     }
   }
 
@@ -401,7 +401,7 @@ public class ReflectiveSchemaTest {
             + " " + fn + " " + name2 + " as c\n"
             + "from \"s\".\"everyTypes\"\n"
             + "where " + name + " <> 0")
-            .returns(Functions.<ResultSet, Void>constantNull());
+            .returns(OptiqAssert.<ResultSet, Void>constantNull());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/mongodb/src/test/java/net/hydromatic/optiq/test/MongoAdapterTest.java
----------------------------------------------------------------------
diff --git a/mongodb/src/test/java/net/hydromatic/optiq/test/MongoAdapterTest.java b/mongodb/src/test/java/net/hydromatic/optiq/test/MongoAdapterTest.java
index bb32774..5ff81c9 100644
--- a/mongodb/src/test/java/net/hydromatic/optiq/test/MongoAdapterTest.java
+++ b/mongodb/src/test/java/net/hydromatic/optiq/test/MongoAdapterTest.java
@@ -17,10 +17,10 @@
 package net.hydromatic.optiq.test;
 
 import net.hydromatic.linq4j.Ord;
-import net.hydromatic.linq4j.function.Function1;
 
 import org.eigenbase.util.*;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
@@ -123,8 +123,8 @@ public class MongoAdapterTest {
 
   /** Returns a function that checks that a particular MongoDB pipeline is
    * generated to implement a query. */
-  private static Function1<List, Void> mongoChecker(final String... strings) {
-    return new Function1<List, Void>() {
+  private static Function<List, Void> mongoChecker(final String... strings) {
+    return new Function<List, Void>() {
       public Void apply(List actual) {
         if (!actual.contains(ImmutableList.copyOf(strings))) {
           Assert.fail("expected MongoDB query not found; actual: " + actual);
@@ -136,9 +136,9 @@ public class MongoAdapterTest {
 
   /** Similar to {@link OptiqAssert#checkResultUnordered}, but filters strings
    * before comparing them. */
-  static Function1<ResultSet, Void> checkResultUnordered(
+  static Function<ResultSet, Void> checkResultUnordered(
       final String... lines) {
-    return new Function1<ResultSet, Void>() {
+    return new Function<ResultSet, Void>() {
       public Void apply(ResultSet resultSet) {
         try {
           final List<String> expectedList = Lists.newArrayList(lines);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/d62c064f/plus/src/test/java/net/hydromatic/optiq/impl/tpch/TpchTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/net/hydromatic/optiq/impl/tpch/TpchTest.java b/plus/src/test/java/net/hydromatic/optiq/impl/tpch/TpchTest.java
index 3036397..222ffe2 100644
--- a/plus/src/test/java/net/hydromatic/optiq/impl/tpch/TpchTest.java
+++ b/plus/src/test/java/net/hydromatic/optiq/impl/tpch/TpchTest.java
@@ -16,14 +16,13 @@
  */
 package net.hydromatic.optiq.impl.tpch;
 
-import net.hydromatic.linq4j.function.Function1;
-
 import net.hydromatic.optiq.test.OptiqAssert;
 
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
 import org.eigenbase.util.Util;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 
 import org.junit.Ignore;
@@ -819,13 +818,14 @@ public class TpchTest {
   @Test public void testQuery02Conversion() {
     query(2, true)
         .enable(ENABLE)
-        .convertMatches(new Function1<RelNode, Void>() {
-          public Void apply(RelNode relNode) {
-            String s = RelOptUtil.toString(relNode);
-            assertThat(s, not(containsString("CorrelatorRel")));
-            return null;
-          }
-        });
+        .convertMatches(
+          new Function<RelNode, Void>() {
+            public Void apply(RelNode relNode) {
+              String s = RelOptUtil.toString(relNode);
+              assertThat(s, not(containsString("CorrelatorRel")));
+              return null;
+            }
+          });
   }
 
   @Test public void testQuery03() {


[4/4] git commit: [OPTIQ-404] MergeProjectRule should not construct RexPrograms for simple mappings

Posted by jh...@apache.org.
[OPTIQ-404] MergeProjectRule should not construct RexPrograms for simple mappings

Move ProjectRel.getPermutation() and .isMapping() up to ProjectRelBase.

Fix a bug in RelOptUtil.projectMapping.


Project: http://git-wip-us.apache.org/repos/asf/incubator-optiq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-optiq/commit/6fbefcec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-optiq/tree/6fbefcec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-optiq/diff/6fbefcec

Branch: refs/heads/master
Commit: 6fbefcec9c21b67577edad9638d4e8299c4971cc
Parents: 25c6c0c
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Sep 5 16:18:14 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 5 16:18:14 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/eigenbase/rel/ProjectRel.java | 36 --------------------
 .../java/org/eigenbase/rel/ProjectRelBase.java  | 36 ++++++++++++++++++++
 .../eigenbase/rel/rules/MergeProjectRule.java   | 24 +++++++++++++
 .../eigenbase/relopt/RelOptMaterialization.java |  2 --
 .../java/org/eigenbase/relopt/RelOptUtil.java   | 10 +++---
 5 files changed, 64 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/6fbefcec/core/src/main/java/org/eigenbase/rel/ProjectRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/ProjectRel.java b/core/src/main/java/org/eigenbase/rel/ProjectRel.java
index d7a5bf1..faec719 100644
--- a/core/src/main/java/org/eigenbase/rel/ProjectRel.java
+++ b/core/src/main/java/org/eigenbase/rel/ProjectRel.java
@@ -21,7 +21,6 @@ import java.util.*;
 import org.eigenbase.relopt.*;
 import org.eigenbase.reltype.*;
 import org.eigenbase.rex.*;
-import org.eigenbase.util.*;
 
 /**
  * <code>ProjectRel</code> is a relational expression which computes a set of
@@ -100,41 +99,6 @@ public final class ProjectRel extends ProjectRelBase {
   public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }
-
-  /**
-   * Returns a permutation, if this projection is merely a permutation of its
-   * input fields, otherwise null.
-   */
-  public Permutation getPermutation() {
-    final int fieldCount = rowType.getFieldList().size();
-    if (fieldCount != getChild().getRowType().getFieldList().size()) {
-      return null;
-    }
-    Permutation permutation = new Permutation(fieldCount);
-    for (int i = 0; i < fieldCount; ++i) {
-      final RexNode exp = exps.get(i);
-      if (exp instanceof RexInputRef) {
-        permutation.set(i, ((RexInputRef) exp).getIndex());
-      } else {
-        return null;
-      }
-    }
-    return permutation;
-  }
-
-  /**
-   * Checks whether this is a functional mapping.
-   * Every output is a source field, but
-   * a source field may appear as zero, one, or more output fields.
-   */
-  public boolean isMapping() {
-    for (RexNode exp : exps) {
-      if (!(exp instanceof RexInputRef)) {
-        return false;
-      }
-    }
-    return true;
-  }
 }
 
 // End ProjectRel.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/6fbefcec/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java b/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
index 83786b4..68c40f9 100644
--- a/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
+++ b/core/src/main/java/org/eigenbase/rel/ProjectRelBase.java
@@ -24,6 +24,7 @@ import org.eigenbase.reltype.*;
 import org.eigenbase.rex.*;
 import org.eigenbase.sql.*;
 import org.eigenbase.util.Pair;
+import org.eigenbase.util.Permutation;
 import org.eigenbase.util.Util;
 import org.eigenbase.util.mapping.MappingType;
 import org.eigenbase.util.mapping.Mappings;
@@ -269,6 +270,41 @@ public abstract class ProjectRelBase extends SingleRel {
     return mapping;
   }
 
+  /**
+   * Returns a permutation, if this projection is merely a permutation of its
+   * input fields, otherwise null.
+   */
+  public Permutation getPermutation() {
+    final int fieldCount = rowType.getFieldList().size();
+    if (fieldCount != getChild().getRowType().getFieldList().size()) {
+      return null;
+    }
+    Permutation permutation = new Permutation(fieldCount);
+    for (int i = 0; i < fieldCount; ++i) {
+      final RexNode exp = exps.get(i);
+      if (exp instanceof RexInputRef) {
+        permutation.set(i, ((RexInputRef) exp).getIndex());
+      } else {
+        return null;
+      }
+    }
+    return permutation;
+  }
+
+  /**
+   * Checks whether this is a functional mapping.
+   * Every output is a source field, but
+   * a source field may appear as zero, one, or more output fields.
+   */
+  public boolean isMapping() {
+    for (RexNode exp : exps) {
+      if (!(exp instanceof RexInputRef)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /** A collection of integer constants that describe the kind of project. */

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/6fbefcec/core/src/main/java/org/eigenbase/rel/rules/MergeProjectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/MergeProjectRule.java b/core/src/main/java/org/eigenbase/rel/rules/MergeProjectRule.java
index 67889aa..78c67af 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/MergeProjectRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/MergeProjectRule.java
@@ -22,6 +22,7 @@ import org.eigenbase.rel.*;
 import org.eigenbase.rel.RelFactories.ProjectFactory;
 import org.eigenbase.relopt.*;
 import org.eigenbase.rex.*;
+import org.eigenbase.util.Permutation;
 
 /**
  * MergeProjectRule merges a {@link ProjectRelBase} into
@@ -72,6 +73,29 @@ public class MergeProjectRule extends RelOptRule {
     ProjectRelBase bottomProject = call.rel(1);
     RexBuilder rexBuilder = topProject.getCluster().getRexBuilder();
 
+    // If one or both projects are permutations, short-circuit the complex logic
+    // of building a RexProgram.
+    final Permutation topPermutation = topProject.getPermutation();
+    if (topPermutation != null) {
+      if (topPermutation.isIdentity()) {
+        // Let RemoveTrivialProjectRule handle this.
+        return;
+      }
+      final Permutation bottomPermutation = bottomProject.getPermutation();
+      if (bottomPermutation != null) {
+        if (bottomPermutation.isIdentity()) {
+          // Let RemoveTrivialProjectRule handle this.
+          return;
+        }
+        final Permutation product = topPermutation.product(bottomPermutation);
+        call.transformTo(
+            RelOptUtil.projectMapping(bottomProject.getChild(),
+                product.inverse(), topProject.getRowType().getFieldNames(),
+                projectFactory));
+        return;
+      }
+    }
+
     // if we're not in force mode and the two projects reference identical
     // inputs, then return and either let FennelRenameRule or
     // RemoveTrivialProjectRule replace the projects

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/6fbefcec/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
index c3bf2a2..2cf1792 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptMaterialization.java
@@ -157,8 +157,6 @@ public class RelOptMaterialization {
             final Table rightTable = rightRelOptTable.unwrap(Table.class);
             if (leftTable instanceof StarTable
                 && ((StarTable) leftTable).tables.contains(rightTable)) {
-              System.out.println("left: " + leftMapping);
-              System.out.println("right: " + rightMapping);
               Mappings.TargetMapping mapping =
                   Mappings.merge(leftMapping,
                       Mappings.offset(rightMapping,

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/6fbefcec/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
index b898bbf..eae8c2f 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
@@ -2771,21 +2771,19 @@ public abstract class RelOptUtil {
     if (mapping.isIdentity()) {
       return rel;
     }
-    final List<String> outputNameList = new ArrayList<String>();
-    final List<RexNode> exprList = new ArrayList<RexNode>();
+    final List<String> outputNameList = Lists.newArrayList();
+    final List<RexNode> exprList = Lists.newArrayList();
     final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
     final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
-    for (int i = 0; i < fields.size(); i++) {
-      exprList.add(rexBuilder.makeInputRef(rel, i));
-    }
     for (int i = 0; i < mapping.getTargetCount(); i++) {
-      int source = mapping.getSource(i);
+      final int source = mapping.getSource(i);
       final RelDataTypeField sourceField = fields.get(source);
       outputNameList.add(
           ((fieldNames == null)
               || (fieldNames.size() <= i)
               || (fieldNames.get(i) == null)) ? sourceField.getName()
               : fieldNames.get(i));
+      exprList.add(rexBuilder.makeInputRef(rel, source));
     }
     return projectFactory.createProject(rel, exprList, outputNameList);
   }


[2/4] git commit: Fix connection leak in SqlRun

Posted by jh...@apache.org.
Fix connection leak in SqlRun


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

Branch: refs/heads/master
Commit: b139ab80b23a74ce940095ef5db029b5aa03714a
Parents: d62c064
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Sep 5 13:14:54 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 5 13:14:54 2014 -0700

----------------------------------------------------------------------
 .../main/java/net/hydromatic/optiq/tools/SqlRun.java | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/b139ab80/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java b/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
index edac068..5a7b505 100644
--- a/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
+++ b/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
@@ -92,6 +92,14 @@ public class SqlRun {
     }
   }
 
+  private void close() throws SQLException {
+    if (connection != null) {
+      Connection c = connection;
+      connection = null;
+      c.close();
+    }
+  }
+
   public void execute(ConnectionFactory connectionFactory) {
     this.connectionFactory = connectionFactory;
     this.printWriter = new PrintWriter(writer);
@@ -99,6 +107,7 @@ public class SqlRun {
       Command command = new Parser().parse();
       try {
         command.execute(execute);
+        close();
       } catch (Exception e) {
         throw new RuntimeException(
             "Error while executing command " + command, e);
@@ -108,7 +117,11 @@ public class SqlRun {
       }
     } finally {
       printWriter.flush();
-      this.connection = null;
+      try {
+        close();
+      } catch (SQLException e) {
+        // ignore
+      }
     }
   }
 


[3/4] git commit: Don't load FoodMartQuerySet unless we have to. It's big.

Posted by jh...@apache.org.
Don't load FoodMartQuerySet unless we have to. It's big.


Project: http://git-wip-us.apache.org/repos/asf/incubator-optiq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-optiq/commit/25c6c0c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-optiq/tree/25c6c0c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-optiq/diff/25c6c0c7

Branch: refs/heads/master
Commit: 25c6c0c74f9c335c0736b10d3f6490e15dc24eb8
Parents: b139ab8
Author: Julian Hyde <jh...@apache.org>
Authored: Fri Sep 5 13:42:33 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 5 13:42:33 2014 -0700

----------------------------------------------------------------------
 .../net/hydromatic/optiq/test/FoodmartTest.java | 21 +++++++++++++++-----
 1 file changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/25c6c0c7/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java b/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
index f4669e3..ff8432c 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/FoodmartTest.java
@@ -23,6 +23,8 @@ import org.eigenbase.util.IntegerIntervalSet;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
+import com.google.common.collect.ImmutableList;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -102,15 +104,24 @@ public class FoodmartTest {
 
   @Parameterized.Parameters(name = "{index}: foodmart({0})={1}")
   public static List<Object[]> getSqls() throws IOException {
-    final String idList = System.getProperty("optiq.ids");
+    String idList = System.getProperty("optiq.ids");
+    if (!OptiqAssert.ENABLE_SLOW && idList == null) {
+      // Avoid loading the query set in a regular test suite run. It burns too
+      // much memory.
+      return ImmutableList.of();
+    }
     final FoodMartQuerySet set = FoodMartQuerySet.instance();
     final List<Object[]> list = new ArrayList<Object[]>();
     if (idList != null) {
-      StringBuilder buf = new StringBuilder();
-      for (int disabledId : DISABLED_IDS) {
-        buf.append(",-").append(disabledId);
+      if (idList.endsWith(",-disabled")) {
+        StringBuilder buf = new StringBuilder(idList);
+        buf.setLength(buf.length() - ",-disabled".length());
+        for (int disabledId : DISABLED_IDS) {
+          buf.append(",-").append(disabledId);
+        }
+        idList = buf.toString();
       }
-      for (Integer id : IntegerIntervalSet.of(idList + buf)) {
+      for (Integer id : IntegerIntervalSet.of(idList)) {
         final FoodmartQuery query1 = set.queries.get(id);
         if (query1 != null) {
           list.add(new Object[] {id /*, query1.sql */});