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 2018/07/13 22:49:02 UTC

[1/4] calcite git commit: [CALCITE-1916] Use Teradata's TPC-DS generator and run tests against TPC-DS at small scale

Repository: calcite
Updated Branches:
  refs/heads/master 6a2af65c9 -> f5cef1606


[CALCITE-1916] Use Teradata's TPC-DS generator and run tests against TPC-DS at small scale

We still use net.hydromatic.tpcds, but only for the queries.


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

Branch: refs/heads/master
Commit: 93b8349895df2183864bc4bd7ef2fafa227b22e9
Parents: 6c34c6b
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Aug 1 16:25:08 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jul 13 11:21:29 2018 -0700

----------------------------------------------------------------------
 plus/pom.xml                                    |  16 ++
 .../calcite/adapter/tpcds/TpcdsSchema.java      | 210 +++++++++++--------
 .../adapter/tpcds/TpcdsSchemaFactory.java       |   7 +-
 .../apache/calcite/adapter/tpcds/TpcdsTest.java | 159 +++++++++++---
 pom.xml                                         |   6 +
 5 files changed, 280 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/93b83498/plus/pom.xml
----------------------------------------------------------------------
diff --git a/plus/pom.xml b/plus/pom.xml
index c169a58..5628b79 100644
--- a/plus/pom.xml
+++ b/plus/pom.xml
@@ -65,6 +65,11 @@ limitations under the License.
       <type>jar</type>
     </dependency>
     <dependency>
+      <groupId>com.teradata.tpcds</groupId>
+      <artifactId>tpcds</artifactId>
+      <type>jar</type>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -102,6 +107,17 @@ limitations under the License.
     <plugins>
       <!-- Sorted by groupId, artifactId. Put versions in
            pluginManagement in the root POM, not here. -->
+
+      <!-- Base is 1.7, but we require 1.8 for com.teradata.tpcds. -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+          <compilerArgument>-Xlint:deprecation</compilerArgument>
+        </configuration>
+      </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/calcite/blob/93b83498/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
index 618324e..91c1e99 100644
--- a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
+++ b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchema.java
@@ -17,10 +17,13 @@
 package org.apache.calcite.adapter.tpcds;
 
 import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.SchemaPlus;
@@ -29,16 +32,20 @@ import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
-import net.hydromatic.tpcds.TpcdsColumn;
-import net.hydromatic.tpcds.TpcdsEntity;
-import net.hydromatic.tpcds.TpcdsTable;
+import com.teradata.tpcds.Results;
+import com.teradata.tpcds.Session;
+import com.teradata.tpcds.column.Column;
+import com.teradata.tpcds.column.ColumnType;
 
-import java.sql.Date;
+import java.math.BigDecimal;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -47,48 +54,53 @@ import java.util.Map;
  * particular scale factor. */
 public class TpcdsSchema extends AbstractSchema {
   private final double scaleFactor;
-  private final int part;
-  private final int partCount;
   private final ImmutableMap<String, Table> tableMap;
 
   // From TPC-DS spec, table 3-2 "Database Row Counts", for 1G sizing.
   private static final ImmutableMap<String, Integer> TABLE_ROW_COUNTS =
       ImmutableMap.<String, Integer>builder()
-          .put("call_center", 8)
-          .put("catalog_page", 11718)
-          .put("catalog_returns", 144067)
-          .put("catalog_sales", 1441548)
-          .put("customer", 100000)
-          .put("customer_address", 50000)
-          .put("customer_demographics", 1920800)
-          .put("date_dim", 73049)
-          .put("household_demographics", 7200)
-          .put("income_band", 20)
-          .put("inventory", 11745000)
-          .put("item", 18000)
-          .put("promotion", 300)
-          .put("reason", 35)
-          .put("ship_mode", 20)
-          .put("store", 12)
-          .put("store_returns", 287514)
-          .put("store_sales", 2880404)
-          .put("time_dim", 86400)
-          .put("warehouse", 5)
-          .put("web_page", 60)
-          .put("web_returns", 71763)
-          .put("web_sales", 719384)
-          .put("web_site", 1)
+          .put("CALL_CENTER", 8)
+          .put("CATALOG_PAGE", 11718)
+          .put("CATALOG_RETURNS", 144067)
+          .put("CATALOG_SALES", 1441548)
+          .put("CUSTOMER", 100000)
+          .put("CUSTOMER_ADDRESS", 50000)
+          .put("CUSTOMER_DEMOGRAPHICS", 1920800)
+          .put("DATE_DIM", 73049)
+          .put("DBGEN_VERSION", 1)
+          .put("HOUSEHOLD_DEMOGRAPHICS", 7200)
+          .put("INCOME_BAND", 20)
+          .put("INVENTORY", 11745000)
+          .put("ITEM", 18000)
+          .put("PROMOTION", 300)
+          .put("REASON", 35)
+          .put("SHIP_MODE", 20)
+          .put("STORE", 12)
+          .put("STORE_RETURNS", 287514)
+          .put("STORE_SALES", 2880404)
+          .put("TIME_DIM", 86400)
+          .put("WAREHOUSE", 5)
+          .put("WEB_PAGE", 60)
+          .put("WEB_RETURNS", 71763)
+          .put("WEB_SALES", 719384)
+          .put("WEB_SITE", 1)
           .build();
 
+  @Deprecated
   public TpcdsSchema(double scaleFactor, int part, int partCount) {
+    this(scaleFactor);
+    Util.discard(part);
+    Util.discard(partCount);
+  }
+
+  /** Creates a TpcdsSchema. */
+  public TpcdsSchema(double scaleFactor) {
     this.scaleFactor = scaleFactor;
-    this.part = part;
-    this.partCount = partCount;
 
     final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
-    for (TpcdsTable<?> tpcdsTable : TpcdsTable.getTables()) {
-      //noinspection unchecked
-      builder.put(tpcdsTable.getTableName().toUpperCase(Locale.ROOT),
+    for (com.teradata.tpcds.Table tpcdsTable
+        : com.teradata.tpcds.Table.getBaseTables()) {
+      builder.put(tpcdsTable.name().toUpperCase(Locale.ROOT),
           new TpcdsQueryableTable(tpcdsTable));
     }
     this.tableMap = builder.build();
@@ -98,22 +110,45 @@ public class TpcdsSchema extends AbstractSchema {
     return tableMap;
   }
 
+  private static Object convert(String string, Column column) {
+    if (string == null) {
+      return null;
+    }
+    switch (column.getType().getBase()) {
+    case IDENTIFIER:
+      return Long.valueOf(string);
+    case INTEGER:
+      return Integer.valueOf(string);
+    case CHAR:
+    case VARCHAR:
+      return string;
+    case DATE:
+      return DateTimeUtils.dateStringToUnixDate(string);
+    case TIME:
+      return DateTimeUtils.timeStringToUnixDate(string);
+    case DECIMAL:
+      return new BigDecimal(string);
+    default:
+      throw new AssertionError(column);
+    }
+  }
+
   /** Definition of a table in the TPC-DS schema.
    *
    * @param <E> entity type */
-  private class TpcdsQueryableTable<E extends TpcdsEntity>
+  private class TpcdsQueryableTable<E extends com.teradata.tpcds.Table>
       extends AbstractQueryableTable {
-    private final TpcdsTable<E> tpcdsTable;
+    private final com.teradata.tpcds.Table tpcdsTable;
 
-    TpcdsQueryableTable(TpcdsTable<E> tpcdsTable) {
+    TpcdsQueryableTable(com.teradata.tpcds.Table tpcdsTable) {
       super(Object[].class);
       this.tpcdsTable = tpcdsTable;
     }
 
     @Override public Statistic getStatistic() {
       Bug.upgrade("add row count estimate to TpcdsTable, and use it");
-      Integer rowCount = TABLE_ROW_COUNTS.get(tpcdsTable.name);
-      assert rowCount != null : tpcdsTable.name;
+      Integer rowCount = TABLE_ROW_COUNTS.get(tpcdsTable.name());
+      assert rowCount != null : tpcdsTable;
       return Statistics.of(rowCount, ImmutableList.of());
     }
 
@@ -123,62 +158,67 @@ public class TpcdsSchema extends AbstractSchema {
       return (Queryable) new AbstractTableQueryable<Object[]>(queryProvider,
           schema, this, tableName) {
         public Enumerator<Object[]> enumerator() {
-          final Enumerator<E> iterator =
-              Linq4j.iterableEnumerator(
-                  tpcdsTable.createGenerator(scaleFactor, part, partCount));
-          return new Enumerator<Object[]>() {
-            public Object[] current() {
-              final List<TpcdsColumn<E>> columns = tpcdsTable.getColumns();
-              final Object[] objects = new Object[columns.size()];
-              int i = 0;
-              for (TpcdsColumn<E> column : columns) {
-                objects[i++] = value(column, iterator.current());
-              }
-              return objects;
-            }
-
-            private Object value(TpcdsColumn<E> tpcdsColumn, E current) {
-              final Class<?> type = realType(tpcdsColumn);
-              if (type == String.class) {
-                return tpcdsColumn.getString(current);
-              } else if (type == Double.class) {
-                return tpcdsColumn.getDouble(current);
-              } else if (type == Date.class) {
-                return Date.valueOf(tpcdsColumn.getString(current));
-              } else {
-                return tpcdsColumn.getLong(current);
-              }
-            }
-
-            public boolean moveNext() {
-              return iterator.moveNext();
-            }
-
-            public void reset() {
-              iterator.reset();
-            }
-
-            public void close() {
-            }
-          };
+          final Session session =
+              Session.getDefaultSession()
+                  .withTable(tpcdsTable)
+                  .withScale(scaleFactor);
+          final Results results = Results.constructResults(tpcdsTable, session);
+          return Linq4j.asEnumerable(results)
+              .selectMany(
+                  new Function1<List<List<String>>, Enumerable<Object[]>>() {
+                    final Column[] columns = tpcdsTable.getColumns();
+
+                    public Enumerable<Object[]> apply(
+                        List<List<String>> inRows) {
+                      final List<Object[]> rows = new ArrayList<>();
+                      for (List<String> strings : inRows) {
+                        final Object[] values = new Object[columns.length];
+                        for (int i = 0; i < strings.size(); i++) {
+                          values[i] = convert(strings.get(i), columns[i]);
+                        }
+                        rows.add(values);
+                      }
+                      return Linq4j.asEnumerable(rows);
+                    }
+
+                  })
+              .enumerator();
         }
       };
     }
 
     public RelDataType getRowType(RelDataTypeFactory typeFactory) {
       final RelDataTypeFactory.Builder builder = typeFactory.builder();
-      for (TpcdsColumn<E> column : tpcdsTable.getColumns()) {
-        builder.add(column.getColumnName().toUpperCase(Locale.ROOT),
-            typeFactory.createJavaType(realType(column)));
+      for (Column column : tpcdsTable.getColumns()) {
+        builder.add(column.getName().toUpperCase(Locale.ROOT),
+            type(typeFactory, column));
       }
       return builder.build();
     }
 
-    private Class<?> realType(TpcdsColumn<E> column) {
-      if (column.getColumnName().endsWith("date")) {
-        return Date.class;
+    private RelDataType type(RelDataTypeFactory typeFactory, Column column) {
+      final ColumnType type = column.getType();
+      switch (type.getBase()) {
+      case DATE:
+        return typeFactory.createSqlType(SqlTypeName.DATE);
+      case TIME:
+        return typeFactory.createSqlType(SqlTypeName.TIME);
+      case INTEGER:
+        return typeFactory.createSqlType(SqlTypeName.INTEGER);
+      case IDENTIFIER:
+        return typeFactory.createSqlType(SqlTypeName.BIGINT);
+      case DECIMAL:
+        return typeFactory.createSqlType(SqlTypeName.DECIMAL,
+            type.getPrecision().get(), type.getScale().get());
+      case VARCHAR:
+        return typeFactory.createSqlType(SqlTypeName.VARCHAR,
+            type.getPrecision().get());
+      case CHAR:
+        return typeFactory.createSqlType(SqlTypeName.CHAR,
+            type.getPrecision().get());
+      default:
+        throw new AssertionError(type.getBase() + ": " + column);
       }
-      return column.getType();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/93b83498/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchemaFactory.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchemaFactory.java b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchemaFactory.java
index 2deb4c0..33021da 100644
--- a/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchemaFactory.java
+++ b/plus/src/main/java/org/apache/calcite/adapter/tpcds/TpcdsSchemaFactory.java
@@ -36,12 +36,9 @@ public class TpcdsSchemaFactory implements SchemaFactory {
 
   public Schema create(SchemaPlus parentSchema, String name,
       Map<String, Object> operand) {
-    Map map = (Map) operand;
+    @SuppressWarnings("RawTypeCanBeGeneric") final Map map = operand;
     double scale = Util.first((Double) map.get("scale"), 1D);
-    int part = Util.first((Integer) map.get("part"), 1);
-    int partCount = Util.first((Integer) map.get("partCount"), 1);
-    boolean columnPrefix = Util.first((Boolean) map.get("columnPrefix"), true);
-    return new TpcdsSchema(scale, part, partCount);
+    return new TpcdsSchema(scale);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/93b83498/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java b/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
index b4c079f..a4ee925 100644
--- a/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
+++ b/plus/src/test/java/org/apache/calcite/adapter/tpcds/TpcdsTest.java
@@ -18,7 +18,6 @@ package org.apache.calcite.adapter.tpcds;
 
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.runtime.Hook;
@@ -32,7 +31,6 @@ import org.apache.calcite.tools.Programs;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
-import org.apache.calcite.util.Pair;
 
 import net.hydromatic.tpcds.query.Query;
 
@@ -49,9 +47,9 @@ import java.util.function.Consumer;
  * command-line.
  * (See {@link org.apache.calcite.test.CalciteAssert#ENABLE_SLOW}.)</p> */
 public class TpcdsTest {
-  private static Consumer<Pair<List<Prepare.Materialization>, Holder<Program>>>
-      handler(boolean bushy, int minJoinCount) {
-    return pair -> pair.right.set(
+  private static Consumer<Holder<Program>> handler(
+      final boolean bushy, final int minJoinCount) {
+    return holder -> holder.set(
         Programs.sequence(
             Programs.heuristicJoinOrder(Programs.RULE_SET, bushy,
                 minJoinCount),
@@ -86,16 +84,117 @@ public class TpcdsTest {
   }
 
   @Test public void testCallCenter() {
-    with()
-        .query("select * from tpcds.call_center")
-        .returnsUnordered();
+    final String[] strings = {
+        "CC_CALL_CENTER_SK=1; CC_CALL_CENTER_ID=AAAAAAAABAAAAAAA; CC_REC_START_DATE=1998-01-01;"
+            + " CC_REC_END_DATE=null; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2450952;"
+            + " CC_NAME=NY Metro; CC_CLASS=large; CC_EMPLOYEES=2; CC_SQ_FT=1138;"
+            + " CC_HOURS=8AM-4PM             ; CC_MANAGER=Bob Belcher; CC_MKT_ID=6;"
+            + " CC_MKT_CLASS=More than other authori                           ;"
+            + " CC_MKT_DESC=Shared others could not count fully dollars. New members ca;"
+            + " CC_MARKET_MANAGER=Julius Tran; CC_DIVISION=3; CC_DIVISION_NAME=pri;"
+            + " CC_COMPANY=6; CC_COMPANY_NAME=cally                                             ;"
+            + " CC_STREET_NUMBER=730       ; CC_STREET_NAME=Ash Hill;"
+            + " CC_STREET_TYPE=Boulevard      ; CC_SUITE_NUMBER=Suite 0   ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.11",
+        "CC_CALL_CENTER_SK=2; CC_CALL_CENTER_ID=AAAAAAAACAAAAAAA; CC_REC_START_DATE=1998-01-01;"
+            + " CC_REC_END_DATE=2000-12-31; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2450806;"
+            + " CC_NAME=Mid Atlantic; CC_CLASS=medium; CC_EMPLOYEES=6; CC_SQ_FT=2268;"
+            + " CC_HOURS=8AM-8AM             ; CC_MANAGER=Felipe Perkins; CC_MKT_ID=2;"
+            + " CC_MKT_CLASS=A bit narrow forms matter animals. Consist        ;"
+            + " CC_MKT_DESC=Largely blank years put substantially deaf, new others. Question;"
+            + " CC_MARKET_MANAGER=Julius Durham; CC_DIVISION=5; CC_DIVISION_NAME=anti;"
+            + " CC_COMPANY=1; CC_COMPANY_NAME=ought                                             ;"
+            + " CC_STREET_NUMBER=984       ; CC_STREET_NAME=Center Hill;"
+            + " CC_STREET_TYPE=Way            ; CC_SUITE_NUMBER=Suite 70  ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.12",
+        "CC_CALL_CENTER_SK=3; CC_CALL_CENTER_ID=AAAAAAAACAAAAAAA; CC_REC_START_DATE=2001-01-01;"
+            + " CC_REC_END_DATE=null; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2450806;"
+            + " CC_NAME=Mid Atlantic; CC_CLASS=medium; CC_EMPLOYEES=6; CC_SQ_FT=4134;"
+            + " CC_HOURS=8AM-4PM             ; CC_MANAGER=Mark Hightower; CC_MKT_ID=2;"
+            + " CC_MKT_CLASS=Wrong troops shall work sometimes in a opti       ;"
+            + " CC_MKT_DESC=Largely blank years put substantially deaf, new others. Question;"
+            + " CC_MARKET_MANAGER=Julius Durham; CC_DIVISION=1; CC_DIVISION_NAME=ought;"
+            + " CC_COMPANY=2; CC_COMPANY_NAME=able                                              ;"
+            + " CC_STREET_NUMBER=984       ; CC_STREET_NAME=Center Hill;"
+            + " CC_STREET_TYPE=Way            ; CC_SUITE_NUMBER=Suite 70  ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.01",
+        "CC_CALL_CENTER_SK=4; CC_CALL_CENTER_ID=AAAAAAAAEAAAAAAA; CC_REC_START_DATE=1998-01-01;"
+            + " CC_REC_END_DATE=2000-01-01; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2451063;"
+            + " CC_NAME=North Midwest; CC_CLASS=medium; CC_EMPLOYEES=1; CC_SQ_FT=649;"
+            + " CC_HOURS=8AM-4PM             ; CC_MANAGER=Larry Mccray; CC_MKT_ID=2;"
+            + " CC_MKT_CLASS=Dealers make most historical, direct students     ;"
+            + " CC_MKT_DESC=Rich groups catch longer other fears; future,;"
+            + " CC_MARKET_MANAGER=Matthew Clifton; CC_DIVISION=4; CC_DIVISION_NAME=ese;"
+            + " CC_COMPANY=3; CC_COMPANY_NAME=pri                                               ;"
+            + " CC_STREET_NUMBER=463       ; CC_STREET_NAME=Pine Ridge;"
+            + " CC_STREET_TYPE=RD             ; CC_SUITE_NUMBER=Suite U   ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.05",
+        "CC_CALL_CENTER_SK=5; CC_CALL_CENTER_ID=AAAAAAAAEAAAAAAA; CC_REC_START_DATE=2000-01-02;"
+            + " CC_REC_END_DATE=2001-12-31; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2451063;"
+            + " CC_NAME=North Midwest; CC_CLASS=small; CC_EMPLOYEES=3; CC_SQ_FT=795;"
+            + " CC_HOURS=8AM-8AM             ; CC_MANAGER=Larry Mccray; CC_MKT_ID=2;"
+            + " CC_MKT_CLASS=Dealers make most historical, direct students     ;"
+            + " CC_MKT_DESC=Blue, due beds come. Politicians would not make far thoughts. "
+            + "Specifically new horses partic;"
+            + " CC_MARKET_MANAGER=Gary Colburn; CC_DIVISION=4; CC_DIVISION_NAME=ese;"
+            + " CC_COMPANY=3; CC_COMPANY_NAME=pri                                               ;"
+            + " CC_STREET_NUMBER=463       ; CC_STREET_NAME=Pine Ridge;"
+            + " CC_STREET_TYPE=RD             ; CC_SUITE_NUMBER=Suite U   ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.12",
+        "CC_CALL_CENTER_SK=6; CC_CALL_CENTER_ID=AAAAAAAAEAAAAAAA; CC_REC_START_DATE=2002-01-01;"
+            + " CC_REC_END_DATE=null; CC_CLOSED_DATE_SK=null; CC_OPEN_DATE_SK=2451063;"
+            + " CC_NAME=North Midwest; CC_CLASS=medium; CC_EMPLOYEES=7; CC_SQ_FT=3514;"
+            + " CC_HOURS=8AM-4PM             ; CC_MANAGER=Larry Mccray; CC_MKT_ID=5;"
+            + " CC_MKT_CLASS=Silly particles could pro                         ;"
+            + " CC_MKT_DESC=Blue, due beds come. Politicians would not make far thoughts. "
+            + "Specifically new horses partic;"
+            + " CC_MARKET_MANAGER=Gary Colburn; CC_DIVISION=5; CC_DIVISION_NAME=anti;"
+            + " CC_COMPANY=3; CC_COMPANY_NAME=pri                                               ;"
+            + " CC_STREET_NUMBER=463       ; CC_STREET_NAME=Pine Ridge;"
+            + " CC_STREET_TYPE=RD             ; CC_SUITE_NUMBER=Suite U   ; CC_CITY=Midway;"
+            + " CC_COUNTY=Williamson County; CC_STATE=TN; CC_ZIP=31904     ;"
+            + " CC_COUNTRY=United States; CC_GMT_OFFSET=-5; CC_TAX_PERCENTAGE=0.11"};
+    with().query("select * from tpcds.call_center").returnsUnordered(strings);
   }
 
-  @Ignore("add tests like this that count each table")
-  @Test public void testLineItem() {
-    with()
-        .query("select * from tpcds.lineitem")
-        .returnsCount(6001215);
+  @Test public void testTableCount() {
+    final CalciteAssert.AssertThat with = with();
+//    foo(with, "CALL_CENTER", 6);
+//    foo(with, "CATALOG_PAGE", 11_718);
+//    foo(with, "CATALOG_RETURNS", 144_067);
+//    foo(with, "CATALOG_SALES", 1_441_548);
+//    foo(with, "CUSTOMER", 100_000);
+//    foo(with, "CUSTOMER_ADDRESS", 50_000);
+//    foo(with, "CUSTOMER_DEMOGRAPHICS", 1_920_800);
+//    foo(with, "DATE_DIM", 73_049);
+//    foo(with, "HOUSEHOLD_DEMOGRAPHICS", 7_200);
+//    foo(with, "INCOME_BAND", 20);
+//    foo(with, "INVENTORY", 11_745_000);
+//    foo(with, "ITEM", 18_000);
+//    foo(with, "PROMOTION", 300);
+//    foo(with, "REASON", 35);
+//    foo(with, "SHIP_MODE", 20);
+//    foo(with, "STORE", 12);
+//    foo(with, "STORE_RETURNS", 287_514);
+//    foo(with, "STORE_SALES", 2_880_404);
+//    foo(with, "TIME_DIM", 86_400);
+//    foo(with, "WAREHOUSE", 5);
+//    foo(with, "WEB_PAGE", 60);
+//    foo(with, "WEB_RETURNS", 71_763);
+//    foo(with, "WEB_SALES", 719_384);
+//    foo(with, "WEB_SITE", 30);
+    foo(with, "DBGEN_VERSION", 1);
+  }
+
+  protected void foo(CalciteAssert.AssertThat with, String tableName,
+      int expectedCount) {
+    final String sql = "select * from tpcds." + tableName;
+    with.query(sql).returnsCount(expectedCount);
   }
 
   /** Tests the customer table with scale factor 5. */
@@ -106,6 +205,7 @@ public class TpcdsTest {
         .returnsCount(750000);
   }
 
+  @Ignore("throws 'RuntimeException: Cannot convert null to long'")
   @Test public void testQuery01() {
     checkQuery(1).runs();
   }
@@ -116,35 +216,37 @@ public class TpcdsTest {
         .withHook(Hook.PROGRAM, handler(true, 2))
         .explainMatches("including all attributes ",
             CalciteAssert.checkMaskedResultContains(""
-                + "EnumerableCalcRel(expr#0..11=[{inputs}], expr#12=[/($t5, $t4)], expr#13=[/($t8, $t7)], expr#14=[/($t11, $t10)], proj#0..5=[{exprs}], STORE_SALES_QUANTITYCOV=[$t12], AS_STORE_RETURNS_QUANTITYCOUNT=[$t6], AS_STORE_RETURNS_QUANTITYAVE=[$t7], AS_STORE_RETURNS_QUANTITYSTDEV=[$t8], STORE_RETURNS_QUANTITYCOV=[$t13], CATALOG_SALES_QUANTITYCOUNT=[$t9], CATALOG_SALES_QUANTITYAVE=[$t10], CATALOG_SALES_QUANTITYSTDEV=[$t14], CATALOG_SALES_QUANTITYCOV=[$t14]): rowcount = 5.434029018852197E26, cumulative cost = {1.618185849567114E30 rows, 1.2672155671963324E30 cpu, 0.0 io}\n"
-                + "  EnumerableSortRel(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC], dir1=[ASC], dir2=[ASC]): rowcount = 5.434029018852197E26, cumulative cost = {1.6176424466652288E30 rows, 1.2509134801397759E30 cpu, 0.0 io}\n"
-                + "    EnumerableCalcRel(expr#0..12=[{inputs}], expr#13=[/($t4, $t5)], expr#14=[CAST($t13):JavaType(class java.lang.Integer)], expr#15=[*($t4, $t4)], expr#16=[/($t15, $t5)], expr#17=[-($t6, $t16)], expr#18=[1], expr#19=[=($t5, $t18)], expr#20=[null], expr#21=[-($t5, $t18)], expr#22=[CASE($t19, $t20, $t21)], expr#23=[/($t17, $t22)], expr#24=[0.5], expr#25=[POWER($t23, $t24)], expr#26=[CAST($t25):JavaType(class java.lang.Integer)], expr#27=[/($t8, $t7)], expr#28=[CAST($t27):JavaType(class java.lang.Integer)], expr#29=[*($t8, $t8)], expr#30=[/($t29, $t7)], expr#31=[-($t9, $t30)], expr#32=[=($t7, $t18)], expr#33=[-($t7, $t18)], expr#34=[CASE($t32, $t20, $t33)], expr#35=[/($t31, $t34)], expr#36=[POWER($t35, $t24)], expr#37=[CAST($t36):JavaType(class java.lang.Integer)], expr#38=[/($t11, $t10)], expr#39=[CAST($t38):JavaType(class java.lang.Integer)], expr#40=[*($t11, $t11)], expr#41=[/($t40, $t10)], expr#42=[-($t12, $t41)], expr#43=[=($t10, $t18)], expr#44=[-($t10, $t18)],
  expr#45=[CASE($t43, $t20, $t44)], expr#46=[/($t42, $t45)], expr#47=[POWER($t46, $t24)], expr#48=[CAST($t47):JavaType(class java.lang.Integer)], proj#0..3=[{exprs}], STORE_SALES_QUANTITYAVE=[$t14], STORE_SALES_QUANTITYSTDEV=[$t26], AS_STORE_RETURNS_QUANTITYCOUNT=[$t7], AS_STORE_RETURNS_QUANTITYAVE=[$t28], AS_STORE_RETURNS_QUANTITYSTDEV=[$t37], CATALOG_SALES_QUANTITYCOUNT=[$t10], CATALOG_SALES_QUANTITYAVE=[$t39], $f11=[$t48]): rowcount = 5.434029018852197E26, cumulative cost = {1.1954863841615548E28 rows, 1.2503700772378907E30 cpu, 0.0 io}\n"
-                + "      EnumerableAggregateRel(group=[{0, 1, 2}], STORE_SALES_QUANTITYCOUNT=[COUNT()], agg#1=[SUM($3)], agg#2=[COUNT($3)], agg#3=[SUM($6)], AS_STORE_RETURNS_QUANTITYCOUNT=[COUNT($4)], agg#5=[SUM($4)], agg#6=[SUM($7)], CATALOG_SALES_QUANTITYCOUNT=[COUNT($5)], agg#8=[SUM($5)], agg#9=[SUM($8)]): rowcount = 5.434029018852197E26, cumulative cost = {1.1411460939730328E28 rows, 1.2172225002228922E30 cpu, 0.0 io}\n"
-                + "        EnumerableCalcRel(expr#0..211=[{inputs}], expr#212=[*($t89, $t89)], expr#213=[*($t140, $t140)], expr#214=[*($t196, $t196)], I_ITEM_ID=[$t58], I_ITEM_DESC=[$t61], S_STATE=[$t24], SS_QUANTITY=[$t89], SR_RETURN_QUANTITY=[$t140], CS_QUANTITY=[$t196], $f6=[$t212], $f7=[$t213], $f8=[$t214]): rowcount = 5.434029018852197E27, cumulative cost = {1.0868058037845108E28 rows, 1.2172225002228922E30 cpu, 0.0 io}\n"
-                + "          EnumerableJoinRel(condition=[AND(=($82, $133), =($81, $132), =($88, $139))], joinType=[inner]): rowcount = 5.434029018852197E27, cumulative cost = {5.434029018992911E27 rows, 1.8579845E7 cpu, 0.0 io}\n"
-                + "            EnumerableJoinRel(condition=[=($0, $86)], joinType=[inner]): rowcount = 2.3008402586892598E13, cumulative cost = {4.8588854672853766E13 rows, 7354409.0 cpu, 0.0 io}\n"
+                + "EnumerableCalc(expr#0..9=[{inputs}], expr#10=[/($t4, $t3)], expr#11=[CAST($t10):INTEGER NOT NULL], expr#12=[*($t4, $t4)], expr#13=[/($t12, $t3)], expr#14=[-($t5, $t13)], expr#15=[1], expr#16=[=($t3, $t15)], expr#17=[null], expr#18=[-($t3, $t15)], expr#19=[CASE($t16, $t17, $t18)], expr#20=[/($t14, $t19)], expr#21=[0.5], expr#22=[POWER($t20, $t21)], expr#23=[CAST($t22):INTEGER NOT NULL], expr#24=[/($t23, $t11)], expr#25=[/($t6, $t3)], expr#26=[CAST($t25):INTEGER NOT NULL], expr#27=[*($t6, $t6)], expr#28=[/($t27, $t3)], expr#29=[-($t7, $t28)], expr#30=[/($t29, $t19)], expr#31=[POWER($t30, $t21)], expr#32=[CAST($t31):INTEGER NOT NULL], expr#33=[/($t32, $t26)], expr#34=[/($t8, $t3)], expr#35=[CAST($t34):INTEGER NOT NULL], expr#36=[*($t8, $t8)], expr#37=[/($t36, $t3)], expr#38=[-($t9, $t37)], expr#39=[/($t38, $t19)], expr#40=[POWER($t39, $t21)], expr#41=[CAST($t40):INTEGER NOT NULL], expr#42=[/($t41, $t35)], proj#0..3=[{exprs}], STORE_SALES_QUANTITYAVE=[$t11], STORE_SAL
 ES_QUANTITYSTDEV=[$t23], STORE_SALES_QUANTITYCOV=[$t24], AS_STORE_RETURNS_QUANTITYCOUNT=[$t3], AS_STORE_RETURNS_QUANTITYAVE=[$t26], AS_STORE_RETURNS_QUANTITYSTDEV=[$t32], STORE_RETURNS_QUANTITYCOV=[$t33], CATALOG_SALES_QUANTITYCOUNT=[$t3], CATALOG_SALES_QUANTITYAVE=[$t35], CATALOG_SALES_QUANTITYSTDEV=[$t42], CATALOG_SALES_QUANTITYCOV=[$t42]): rowcount = 100.0, cumulative cost = {1.2435775409784036E28 rows, 2.555295485909236E30 cpu, 0.0 io}\n"
+                + "  EnumerableLimit(fetch=[100]): rowcount = 100.0, cumulative cost = {1.2435775409784036E28 rows, 2.555295485909236E30 cpu, 0.0 io}\n"
+                + "    EnumerableSort(sort0=[$0], sort1=[$1], sort2=[$2], dir0=[ASC], dir1=[ASC], dir2=[ASC]): rowcount = 5.434029018852197E26, cumulative cost = {1.2435775409784036E28 rows, 2.555295485909236E30 cpu, 0.0 io}\n"
+                + "      EnumerableAggregate(group=[{0, 1, 2}], STORE_SALES_QUANTITYCOUNT=[COUNT()], agg#1=[$SUM0($3)], agg#2=[$SUM0($6)], agg#3=[$SUM0($4)], agg#4=[$SUM0($7)], agg#5=[$SUM0($5)], agg#6=[$SUM0($8)]): rowcount = 5.434029018852197E26, cumulative cost = {1.1892372507898816E28 rows, 1.2172225002228922E30 cpu, 0.0 io}\n"
+                + "        EnumerableCalc(expr#0..211=[{inputs}], expr#212=[*($t89, $t89)], expr#213=[*($t140, $t140)], expr#214=[*($t196, $t196)], I_ITEM_ID=[$t58], I_ITEM_DESC=[$t61], S_STATE=[$t24], SS_QUANTITY=[$t89], SR_RETURN_QUANTITY=[$t140], CS_QUANTITY=[$t196], $f6=[$t212], $f7=[$t213], $f8=[$t214]): rowcount = 5.434029018852197E27, cumulative cost = {1.0873492066864028E28 rows, 1.2172225002228922E30 cpu, 0.0 io}\n"
+                + "          EnumerableJoin(condition=[AND(=($82, $133), =($81, $132), =($88, $139))], joinType=[inner]): rowcount = 5.434029018852197E27, cumulative cost = {5.439463048011832E27 rows, 1.8506796E7 cpu, 0.0 io}\n"
+                + "            EnumerableJoin(condition=[=($0, $86)], joinType=[inner]): rowcount = 2.3008402586892598E13, cumulative cost = {4.8588854672854766E13 rows, 7281360.0 cpu, 0.0 io}\n"
                 + "              EnumerableTableScan(table=[[TPCDS, STORE]]): rowcount = 12.0, cumulative cost = {12.0 rows, 13.0 cpu, 0.0 io}\n"
-                + "              EnumerableJoinRel(condition=[=($0, $50)], joinType=[inner]): rowcount = 1.2782445881607E13, cumulative cost = {1.279800620431234E13 rows, 7354396.0 cpu, 0.0 io}\n"
-                + "                EnumerableCalcRel(expr#0..27=[{inputs}], expr#28=[CAST($t15):VARCHAR(6) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], expr#29=['1998Q1'], expr#30=[=($t28, $t29)], proj#0..27=[{exprs}], $condition=[$t30]): rowcount = 10957.35, cumulative cost = {84006.35 rows, 4455990.0 cpu, 0.0 io}\n"
+                + "              EnumerableJoin(condition=[=($0, $50)], joinType=[inner]): rowcount = 1.2782445881607E13, cumulative cost = {1.279800620431234E13 rows, 7281347.0 cpu, 0.0 io}\n"
+                + "                EnumerableCalc(expr#0..27=[{inputs}], expr#28=['1998Q1'], expr#29=[=($t15, $t28)], proj#0..27=[{exprs}], $condition=[$t29]): rowcount = 10957.35, cumulative cost = {84006.35 rows, 4382941.0 cpu, 0.0 io}\n"
                 + "                  EnumerableTableScan(table=[[TPCDS, DATE_DIM]]): rowcount = 73049.0, cumulative cost = {73049.0 rows, 73050.0 cpu, 0.0 io}\n"
-                + "                EnumerableJoinRel(condition=[=($0, $24)], joinType=[inner]): rowcount = 7.7770908E9, cumulative cost = {7.783045975286664E9 rows, 2898406.0 cpu, 0.0 io}\n"
+                + "                EnumerableJoin(condition=[=($0, $24)], joinType=[inner]): rowcount = 7.7770908E9, cumulative cost = {7.783045975286664E9 rows, 2898406.0 cpu, 0.0 io}\n"
                 + "                  EnumerableTableScan(table=[[TPCDS, ITEM]]): rowcount = 18000.0, cumulative cost = {18000.0 rows, 18001.0 cpu, 0.0 io}\n"
                 + "                  EnumerableTableScan(table=[[TPCDS, STORE_SALES]]): rowcount = 2880404.0, cumulative cost = {2880404.0 rows, 2880405.0 cpu, 0.0 io}\n"
-                + "            EnumerableJoinRel(condition=[AND(=($31, $79), =($30, $91))], joinType=[inner]): rowcount = 6.9978029381741304E16, cumulative cost = {6.9978054204658736E16 rows, 1.1225436E7 cpu, 0.0 io}\n"
-                + "              EnumerableJoinRel(condition=[=($0, $28)], joinType=[inner]): rowcount = 7.87597881975E8, cumulative cost = {7.884434222216867E8 rows, 5035701.0 cpu, 0.0 io}\n"
-                + "                EnumerableCalcRel(expr#0..27=[{inputs}], expr#28=['1998Q1'], expr#29=[=($t15, $t28)], expr#30=['1998Q2'], expr#31=[=($t15, $t30)], expr#32=['1998Q3'], expr#33=[=($t15, $t32)], expr#34=[OR($t29, $t31, $t33)], proj#0..27=[{exprs}], $condition=[$t34]): rowcount = 18262.25, cumulative cost = {91311.25 rows, 4748186.0 cpu, 0.0 io}\n"
+                + "            EnumerableJoin(condition=[AND(=($31, $79), =($30, $91))], joinType=[inner]): rowcount = 6.9978029381741304E16, cumulative cost = {7.0048032234040472E16 rows, 1.1225436E7 cpu, 0.0 io}\n"
+                + "              EnumerableJoin(condition=[=($0, $28)], joinType=[inner]): rowcount = 7.87597881975E8, cumulative cost = {7.884434212216867E8 rows, 5035701.0 cpu, 0.0 io}\n"
+                + "                EnumerableCalc(expr#0..27=[{inputs}], expr#28=['1998Q1'], expr#29=[=($t15, $t28)], expr#30=['1998Q2'], expr#31=[=($t15, $t30)], expr#32=['1998Q3'], expr#33=[=($t15, $t32)], expr#34=[OR($t29, $t31, $t33)], proj#0..27=[{exprs}], $condition=[$t34]): rowcount = 18262.25, cumulative cost = {91311.25 rows, 4748186.0 cpu, 0.0 io}\n"
                 + "                  EnumerableTableScan(table=[[TPCDS, DATE_DIM]]): rowcount = 73049.0, cumulative cost = {73049.0 rows, 73050.0 cpu, 0.0 io}\n"
                 + "                EnumerableTableScan(table=[[TPCDS, STORE_RETURNS]]): rowcount = 287514.0, cumulative cost = {287514.0 rows, 287515.0 cpu, 0.0 io}\n"
-                + "              EnumerableJoinRel(condition=[=($0, $28)], joinType=[inner]): rowcount = 3.94888649445E9, cumulative cost = {3.9520401026966867E9 rows, 6189735.0 cpu, 0.0 io}\n"
-                + "                EnumerableCalcRel(expr#0..27=[{inputs}], expr#28=['1998Q1'], expr#29=[=($t15, $t28)], expr#30=['1998Q2'], expr#31=[=($t15, $t30)], expr#32=['1998Q3'], expr#33=[=($t15, $t32)], expr#34=[OR($t29, $t31, $t33)], proj#0..27=[{exprs}], $condition=[$t34]): rowcount = 18262.25, cumulative cost = {91311.25 rows, 4748186.0 cpu, 0.0 io}\n"
+                + "              EnumerableJoin(condition=[=($0, $28)], joinType=[inner]): rowcount = 3.94888649445E9, cumulative cost = {3.9520401026966867E9 rows, 6189735.0 cpu, 0.0 io}\n"
+                + "                EnumerableCalc(expr#0..27=[{inputs}], expr#28=['1998Q1'], expr#29=[=($t15, $t28)], expr#30=['1998Q2'], expr#31=[=($t15, $t30)], expr#32=['1998Q3'], expr#33=[=($t15, $t32)], expr#34=[OR($t29, $t31, $t33)], proj#0..27=[{exprs}], $condition=[$t34]): rowcount = 18262.25, cumulative cost = {91311.25 rows, 4748186.0 cpu, 0.0 io}\n"
                 + "                  EnumerableTableScan(table=[[TPCDS, DATE_DIM]]): rowcount = 73049.0, cumulative cost = {73049.0 rows, 73050.0 cpu, 0.0 io}\n"
                 + "                EnumerableTableScan(table=[[TPCDS, CATALOG_SALES]]): rowcount = 1441548.0, cumulative cost = {1441548.0 rows, 1441549.0 cpu, 0.0 io}\n"));
   }
 
+  @Ignore("throws 'RuntimeException: Cannot convert null to long'")
   @Test public void testQuery27() {
     checkQuery(27).runs();
   }
 
+  @Ignore("throws 'RuntimeException: Cannot convert null to long'")
   @Test public void testQuery58() {
     checkQuery(58).explainContains("PLAN").runs();
   }
@@ -161,6 +263,7 @@ public class TpcdsTest {
         .planContains("xx");
   }
 
+  @Ignore("throws 'java.lang.AssertionError: type mismatch'")
   @Test public void testQuery95() {
     checkQuery(95)
         .withHook(Hook.PROGRAM, handler(false, 6))

http://git-wip-us.apache.org/repos/asf/calcite/blob/93b83498/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 60d4daa..82cc6ff 100644
--- a/pom.xml
+++ b/pom.xml
@@ -130,6 +130,7 @@ limitations under the License.
     <log4j2.version>2.11.0</log4j2.version>
     <spark.version>2.2.0</spark.version>
     <sqlline.version>1.4.0</sqlline.version>
+    <tpcds.version>1.2</tpcds.version>
     <xalan.version>2.7.1</xalan.version>
     <xerces.version>2.9.1</xerces.version>
     <sketches.version>0.9.0</sketches.version>
@@ -297,6 +298,11 @@ limitations under the License.
         <version>${sketches.version}</version>
       </dependency>
       <dependency>
+        <groupId>com.teradata.tpcds</groupId>
+        <artifactId>tpcds</artifactId>
+        <version>${tpcds.version}</version>
+      </dependency>
+      <dependency>
         <groupId>javax.servlet</groupId>
         <artifactId>javax.servlet-api</artifactId>
         <version>${servlet.version}</version>


[2/4] calcite git commit: [CALCITE-2376] License header for XML file and javadoc (Andrei Sereda)

Posted by jh...@apache.org.
[CALCITE-2376] License header for XML file and javadoc (Andrei Sereda)

Close apache/calcite#758


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

Branch: refs/heads/master
Commit: 6c34c6b94a849447ba72911f0e09f876ce61628a
Parents: 6a2af65
Author: Andrei Sereda <an...@nospam.com>
Authored: Wed Jul 11 20:11:53 2018 -0400
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jul 13 11:21:29 2018 -0700

----------------------------------------------------------------------
 .../EmbeddedElasticsearchNode.java              |  6 ++--
 .../EmbeddedElasticsearchPolicy.java            | 10 +++---
 elasticsearch/src/test/resources/log4j2.xml     | 36 +++++++++-----------
 3 files changed, 24 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/6c34c6b9/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchNode.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchNode.java b/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchNode.java
index 5dfb1f6..54f06c0 100644
--- a/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchNode.java
+++ b/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchNode.java
@@ -38,7 +38,8 @@ import java.util.Objects;
 
 /**
  * Represents a single elastic search node which can run embedded in a java application.
- * Intended for unit and integration tests. Settings and plugins are crafted for Calcite.
+ *
+ * <p>Intended for unit and integration tests. Settings and plugins are crafted for Calcite.
  */
 class EmbeddedElasticsearchNode implements AutoCloseable {
 
@@ -49,7 +50,6 @@ class EmbeddedElasticsearchNode implements AutoCloseable {
     this.node = Objects.requireNonNull(node, "node");
   }
 
-
   /**
    * Creates an instance with existing settings
    * @param settings configuration parameters of ES instance
@@ -116,8 +116,8 @@ class EmbeddedElasticsearchNode implements AutoCloseable {
   /**
    * Exposes elastic
    * <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/transport-client.html">transport client</a>
-   *
    * (use of HTTP client is preferred).
+   *
    * @return current elastic search client
    */
   public Client client() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/6c34c6b9/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchPolicy.java
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchPolicy.java b/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchPolicy.java
index 1ef9175..9a2f69e 100644
--- a/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchPolicy.java
+++ b/elasticsearch/src/test/java/org/apache/calcite/adapter/elasticsearch/EmbeddedElasticsearchPolicy.java
@@ -32,7 +32,7 @@ import java.util.Objects;
  * Used to initialize a single elastic node. For performance reasons (node startup costs),
  * same instance is usually shared across multiple tests.
  *
- * This rule should be used as follows:
+ * <p>This rule should be used as follows:
  * <pre>
  *  public class MyTest {
  *    &#64;ClassRule
@@ -45,7 +45,7 @@ import java.util.Objects;
  *
  *    &#64;Test
  *    public void myTest() {
- *      TransportAddress address = RULE.httpAddress();
+ *      RestClient client = RULE.restClient();
  *      // ....
  *    }
  *  }
@@ -91,8 +91,8 @@ class EmbeddedElasticsearchPolicy extends ExternalResource {
   }
 
   /**
-   * Low-level http rest client to elastic search
-   * @return current ES rest client
+   * Low-level http rest client connected to current embedded elastic search instance.
+   * @return http client connected to ES cluster
    */
   RestClient restClient() {
     if (client != null) {
@@ -110,7 +110,7 @@ class EmbeddedElasticsearchPolicy extends ExternalResource {
    * HTTP address for rest clients (can be ES native or any other).
    * @return http address to connect to
    */
-  TransportAddress httpAddress() {
+  private TransportAddress httpAddress() {
     return node.httpAddress();
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/6c34c6b9/elasticsearch/src/test/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/elasticsearch/src/test/resources/log4j2.xml b/elasticsearch/src/test/resources/log4j2.xml
index c8d70aa..3e28263 100644
--- a/elasticsearch/src/test/resources/log4j2.xml
+++ b/elasticsearch/src/test/resources/log4j2.xml
@@ -1,26 +1,22 @@
 <?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.
-  ~  */
-  ~
-  -->
+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
 
-<Configuration status="WARN">
+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.
+-->
 
+<Configuration status="WARN">
   <Appenders>
     <Console name="console" target="SYSTEM_OUT">
       <PatternLayout
@@ -33,4 +29,4 @@
       <AppenderRef ref="console" />
     </Root>
   </Loggers>
-</Configuration>
\ No newline at end of file
+</Configuration>


[4/4] calcite git commit: Util.filter replaces Guava Iterables.filter

Posted by jh...@apache.org.
Util.filter replaces Guava Iterables.filter


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

Branch: refs/heads/master
Commit: f5cef16066a8f13f906eed495f62b6c12cc1648d
Parents: 4402f91
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Jul 12 12:17:58 2018 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jul 13 14:21:43 2018 -0700

----------------------------------------------------------------------
 .../calcite/plan/AbstractRelOptPlanner.java     |  4 +-
 .../calcite/rel/logical/LogicalWindow.java      |  4 +-
 .../java/org/apache/calcite/rex/RexUtil.java    |  2 +-
 .../sql/type/SqlTypeExplicitPrecedenceList.java |  6 +--
 .../main/java/org/apache/calcite/util/Util.java | 50 ++++++++++++++++++++
 .../java/org/apache/calcite/util/UtilTest.java  | 46 ++++++++++++++++++
 6 files changed, 103 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
index 06306d5..9eab16e 100644
--- a/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/AbstractRelOptPlanner.java
@@ -22,9 +22,9 @@ import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexExecutor;
 import org.apache.calcite.util.CancelFlag;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 
 import java.util.Arrays;
 import java.util.HashMap;
@@ -415,7 +415,7 @@ public abstract class AbstractRelOptPlanner implements RelOptPlanner {
   /** Returns sub-classes of relational expression. */
   public Iterable<Class<? extends RelNode>> subClasses(
       final Class<? extends RelNode> clazz) {
-    return Iterables.filter(classes, clazz::isAssignableFrom);
+    return Util.filter(classes, clazz::isAssignableFrom);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
index f03cc2f..db5b856 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -37,8 +37,8 @@ import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import com.google.common.collect.Iterables;
 import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
@@ -345,7 +345,7 @@ public final class LogicalWindow extends Window {
     // Look up or create a window.
     RelCollation orderKeys = getCollation(
         Lists.newArrayList(
-            Iterables.filter(aggWindow.orderKeys,
+            Util.filter(aggWindow.orderKeys,
                 rexFieldCollation ->
                     // If ORDER BY references constant (i.e. RexInputRef),
                     // then we can ignore such ORDER BY key.

http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 29e63f1..5c69573 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -1855,7 +1855,7 @@ public class RexUtil {
     case EQUALS:
       final RexCall call = (RexCall) e;
       if (call.getOperands().get(1) instanceof RexLiteral) {
-        notTerms = Iterables.filter(notTerms,
+        notTerms = Util.filter(notTerms,
             e2 -> {
               switch (e2.getKind()) {
               case EQUALS:

http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
index 539f058..d8eabdd 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeExplicitPrecedenceList.java
@@ -22,14 +22,13 @@ import org.apache.calcite.util.Glossary;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
 
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * SqlTypeExplicitPrecedenceList implements the
@@ -60,8 +59,7 @@ public class SqlTypeExplicitPrecedenceList
           SqlTypeName.DOUBLE);
 
   private static final List<SqlTypeName> COMPACT_NUMERIC_TYPES =
-      ImmutableList.copyOf(
-          Iterables.filter(NUMERIC_TYPES, Predicates.notNull()));
+      ImmutableList.copyOf(Util.filter(NUMERIC_TYPES, Objects::nonNull));
 
   /**
    * Map from SqlTypeName to corresponding precedence list.

http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index b90bbbd..1139643 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -95,6 +95,7 @@ import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.TimeZone;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.jar.JarFile;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -2369,6 +2370,18 @@ public class Util {
     }
   }
 
+  /** Filters an iterable. */
+  public static <E> Iterable<E> filter(Iterable<E> iterable,
+      Predicate<E> predicate) {
+    return () -> filter(iterable.iterator(), predicate);
+  }
+
+  /** Filters an iterator. */
+  public static <E> Iterator<E> filter(Iterator<E> iterator,
+      Predicate<E> predicate) {
+    return new FilteringIterator<>(iterator, predicate);
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -2447,6 +2460,43 @@ public class Util {
       super(list, function);
     }
   }
+
+  /** Iterator that applies a predicate to each element.
+   *
+   * @param <T> Element type */
+  private static class FilteringIterator<T> implements Iterator<T> {
+    private static final Object DUMMY = new Object();
+    final Iterator<? extends T> iterator;
+    private final Predicate<T> predicate;
+    T current;
+
+    FilteringIterator(Iterator<? extends T> iterator,
+        Predicate<T> predicate) {
+      this.iterator = iterator;
+      this.predicate = predicate;
+      current = moveNext();
+    }
+
+    public boolean hasNext() {
+      return current != DUMMY;
+    }
+
+    public T next() {
+      final T t = this.current;
+      current = moveNext();
+      return t;
+    }
+
+    protected T moveNext() {
+      while (iterator.hasNext()) {
+        T t = iterator.next();
+        if (predicate.test(t)) {
+          return t;
+        }
+      }
+      return (T) DUMMY;
+    }
+  }
 }
 
 // End Util.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f5cef160/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index e9159a9..fa649c8 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -43,8 +43,10 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Ints;
 
+import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.StringDescription;
+import org.hamcrest.TypeSafeMatcher;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -73,6 +75,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.NavigableSet;
+import java.util.Objects;
 import java.util.Properties;
 import java.util.Random;
 import java.util.RandomAccess;
@@ -2175,6 +2178,49 @@ public class UtilTest {
         not(instanceOf(RandomAccess.class)));
   }
 
+  /** Tests {@link Util#filter(Iterable, java.util.function.Predicate)}. */
+  @Test public void testFilter() {
+    final List<String> beatles =
+        Arrays.asList("John", "Paul", "George", "Ringo");
+    final List<String> empty = Collections.emptyList();
+    final List<String> nullBeatles =
+        Arrays.asList("John", "Paul", null, "Ringo");
+    assertThat(Util.filter(beatles, s -> s.length() == 4),
+        isIterable(Arrays.asList("John", "Paul")));
+    assertThat(Util.filter(empty, s -> s.length() == 4), isIterable(empty));
+    assertThat(Util.filter(empty, s -> false), isIterable(empty));
+    assertThat(Util.filter(empty, s -> true), isIterable(empty));
+    assertThat(Util.filter(beatles, s -> false), isIterable(empty));
+    assertThat(Util.filter(beatles, s -> true), isIterable(beatles));
+    assertThat(Util.filter(nullBeatles, s -> false), isIterable(empty));
+    assertThat(Util.filter(nullBeatles, s -> true), isIterable(nullBeatles));
+    assertThat(Util.filter(nullBeatles, Objects::isNull),
+        isIterable(Collections.singletonList(null)));
+    assertThat(Util.filter(nullBeatles, Objects::nonNull),
+        isIterable(Arrays.asList("John", "Paul", "Ringo")));
+  }
+
+  private static <E> Matcher<Iterable<E>> isIterable(final Iterable<E> iterable) {
+    final List<E> list = toList(iterable);
+    return new TypeSafeMatcher<Iterable<E>>() {
+      protected boolean matchesSafely(Iterable<E> iterable) {
+        return list.equals(toList(iterable));
+      }
+
+      public void describeTo(Description description) {
+        description.appendText("is iterable ").appendValue(list);
+      }
+    };
+  }
+
+  private static <E> List<E> toList(Iterable<E> iterable) {
+    final List<E> list = new ArrayList<>();
+    for (E e : iterable) {
+      list.add(e);
+    }
+    return list;
+  }
+
   static String mismatchDescription(Matcher m, Object item) {
     final StringDescription d = new StringDescription();
     m.describeMismatch(item, d);


[3/4] calcite git commit: [CALCITE-1167] OVERLAPS should match even if operands are in (high, low) order

Posted by jh...@apache.org.
[CALCITE-1167] OVERLAPS should match even if operands are in (high, low) order

Already works (probably fixed in [CALCITE-715]), but added a test case.


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

Branch: refs/heads/master
Commit: 4402f9194291def0de7263a57889572df339fdb7
Parents: 93b8349
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Aug 3 21:16:14 2017 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jul 13 14:21:43 2018 -0700

----------------------------------------------------------------------
 core/src/test/resources/sql/misc.iq | 23 +++++++++++++++++++++++
 1 file changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4402f919/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 22d1422..cdbde5d 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -2158,4 +2158,27 @@ where false;
 
 !ok
 
+!set outputformat csv
+
+# [CALCITE-1167] OVERLAPS should match even if operands are in (high, low) order
+values ((date '1999-12-01', date '2001-12-31') overlaps (date '2001-01-01' , date '2002-11-11'));
+EXPR$0
+true
+!ok
+
+values ((date '2001-12-31', date '1999-12-01') overlaps (date '2001-01-01' , date '2002-11-11'));
+EXPR$0
+true
+!ok
+
+values ((date '2001-12-31', date '1999-12-01') overlaps (date '2002-11-11', date '2001-01-01'));
+EXPR$0
+true
+!ok
+
+values ((date '2001-12-31', date '1999-12-01') overlaps (date '2002-01-01', date '2002-11-11'));
+EXPR$0
+false
+!ok
+
 # End misc.iq