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 2019/07/15 16:38:58 UTC

[calcite] 02/06: [CALCITE-3196] In Frameworks, add interface BasePrepareAction (a functional interface) and deprecate abstract class PrepareAction

This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 1c5de1cd27a0251ac28a503e877ef9c21e61b620
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Wed Jul 10 17:26:26 2019 -0700

    [CALCITE-3196] In Frameworks, add interface BasePrepareAction (a functional interface) and deprecate abstract class PrepareAction
---
 .../apache/calcite/prepare/CalcitePrepareImpl.java | 17 +++++--
 .../java/org/apache/calcite/tools/Frameworks.java  | 59 +++++++++++++++-------
 .../java/org/apache/calcite/tools/RelBuilder.java  | 17 ++-----
 .../org/apache/calcite/rex/RexExecutorTest.java    | 58 +++++++++------------
 .../calcite/test/RexImplicationCheckerTest.java    | 26 ++--------
 .../org/apache/calcite/tools/FrameworksTest.java   | 32 +++++-------
 6 files changed, 96 insertions(+), 113 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 7dc1109..7faead7 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -103,6 +103,7 @@ import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
+import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
@@ -884,15 +885,21 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     return typeFactory.builder().add("$0", type).build();
   }
 
-  /** Executes a prepare action. */
+  @Deprecated // to be removed before 2.0
   public <R> R perform(CalciteServerStatement statement,
       Frameworks.PrepareAction<R> action) {
+    return perform(statement, action.getConfig(), action);
+  }
+
+  /** Executes a prepare action. */
+  public <R> R perform(CalciteServerStatement statement,
+      FrameworkConfig config, Frameworks.BasePrepareAction<R> action) {
     final CalcitePrepare.Context prepareContext =
         statement.createPrepareContext();
     final JavaTypeFactory typeFactory = prepareContext.getTypeFactory();
     final CalciteSchema schema =
-        action.getConfig().getDefaultSchema() != null
-            ? CalciteSchema.from(action.getConfig().getDefaultSchema())
+        config.getDefaultSchema() != null
+            ? CalciteSchema.from(config.getDefaultSchema())
             : prepareContext.getRootSchema();
     CalciteCatalogReader catalogReader =
         new CalciteCatalogReader(schema.root(),
@@ -902,8 +909,8 @@ public class CalcitePrepareImpl implements CalcitePrepare {
     final RexBuilder rexBuilder = new RexBuilder(typeFactory);
     final RelOptPlanner planner =
         createPlanner(prepareContext,
-            action.getConfig().getContext(),
-            action.getConfig().getCostFactory());
+            config.getContext(),
+            config.getCostFactory());
     final RelOptCluster cluster = createCluster(planner, rexBuilder);
     return action.apply(cluster, catalogReader,
         prepareContext.getRootSchema().plus(), statement);
diff --git a/core/src/main/java/org/apache/calcite/tools/Frameworks.java b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
index c1f0a31..f4c8875 100644
--- a/core/src/main/java/org/apache/calcite/tools/Frameworks.java
+++ b/core/src/main/java/org/apache/calcite/tools/Frameworks.java
@@ -72,6 +72,7 @@ public class Frameworks {
    * other useful objects.
    *
    * @param <R> result type */
+  @FunctionalInterface
   public interface PlannerAction<R> {
     R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
         SchemaPlus rootSchema);
@@ -84,10 +85,22 @@ public class Frameworks {
    * statement.
    *
    * @param <R> result type */
-  public abstract static class PrepareAction<R> {
+  @FunctionalInterface
+  public interface BasePrepareAction<R> {
+    R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
+        SchemaPlus rootSchema, CalciteServerStatement statement);
+  }
+
+  /** As {@link BasePrepareAction} but with a {@link FrameworkConfig} included.
+   * Deprecated because a functional interface is more convenient.
+   *
+   * @param <R> result type */
+  @Deprecated // to be removed before 2.0
+  public abstract static class PrepareAction<R>
+      implements BasePrepareAction<R> {
     private final FrameworkConfig config;
     public PrepareAction() {
-      this.config = newConfigBuilder() //
+      this.config = newConfigBuilder()
           .defaultSchema(Frameworks.createRootSchema(true)).build();
     }
 
@@ -98,9 +111,6 @@ public class Frameworks {
     public FrameworkConfig getConfig() {
       return config;
     }
-
-    public abstract R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-        SchemaPlus rootSchema, CalciteServerStatement statement);
   }
 
   /**
@@ -110,17 +120,14 @@ public class Frameworks {
    * @param config FrameworkConfig to use for planner action.
    * @return Return value from action
    */
-  public static <R> R withPlanner(final PlannerAction<R> action, //
+  public static <R> R withPlanner(final PlannerAction<R> action,
       final FrameworkConfig config) {
-    return withPrepare(
-        new Frameworks.PrepareAction<R>(config) {
-          public R apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-              SchemaPlus rootSchema, CalciteServerStatement statement) {
-            final CalciteSchema schema =
-                CalciteSchema.from(
-                    Util.first(config.getDefaultSchema(), rootSchema));
-            return action.apply(cluster, relOptSchema, schema.root().plus());
-          }
+    return withPrepare(config,
+        (cluster, relOptSchema, rootSchema, statement) -> {
+          final CalciteSchema schema =
+              CalciteSchema.from(
+                  Util.first(config.getDefaultSchema(), rootSchema));
+          return action.apply(cluster, relOptSchema, schema.root().plus());
         });
   }
 
@@ -136,6 +143,19 @@ public class Frameworks {
     return withPlanner(action, config);
   }
 
+  @Deprecated // to be removed before 2.0
+  public static <R> R withPrepare(PrepareAction<R> action) {
+    return withPrepare(action.getConfig(), action);
+  }
+
+  /** As {@link #withPrepare(FrameworkConfig, BasePrepareAction)} but using a
+   * default configuration. */
+  public static <R> R withPrepare(BasePrepareAction<R> action) {
+    final FrameworkConfig config = newConfigBuilder()
+        .defaultSchema(Frameworks.createRootSchema(true)).build();
+    return withPrepare(config, action);
+  }
+
   /**
    * Initializes a container then calls user-specified code with a planner
    * and statement.
@@ -143,19 +163,20 @@ public class Frameworks {
    * @param action Callback containing user-specified code
    * @return Return value from action
    */
-  public static <R> R withPrepare(PrepareAction<R> action) {
+  public static <R> R withPrepare(FrameworkConfig config,
+      BasePrepareAction<R> action) {
     try {
       final Properties info = new Properties();
-      if (action.config.getTypeSystem() != RelDataTypeSystem.DEFAULT) {
+      if (config.getTypeSystem() != RelDataTypeSystem.DEFAULT) {
         info.setProperty(CalciteConnectionProperty.TYPE_SYSTEM.camelName(),
-            action.config.getTypeSystem().getClass().getName());
+            config.getTypeSystem().getClass().getName());
       }
       Connection connection =
           DriverManager.getConnection("jdbc:calcite:", info);
       final CalciteServerStatement statement =
           connection.createStatement()
               .unwrap(CalciteServerStatement.class);
-      return new CalcitePrepareImpl().perform(statement, action);
+      return new CalcitePrepareImpl().perform(statement, config, action);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 7058e50..983fa35 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -70,10 +70,8 @@ import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.runtime.Hook;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.TransientTable;
 import org.apache.calcite.schema.impl.ListTransientTable;
-import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -225,18 +223,9 @@ public class RelBuilder {
 
   /** Creates a RelBuilder. */
   public static RelBuilder create(FrameworkConfig config) {
-    final RelOptCluster[] clusters = {null};
-    final RelOptSchema[] relOptSchemas = {null};
-    Frameworks.withPrepare(
-        new Frameworks.PrepareAction<Void>(config) {
-          public Void apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-              SchemaPlus rootSchema, CalciteServerStatement statement) {
-            clusters[0] = cluster;
-            relOptSchemas[0] = relOptSchema;
-            return null;
-          }
-        });
-    return new RelBuilder(config.getContext(), clusters[0], relOptSchemas[0]);
+    return Frameworks.withPrepare(config,
+        (cluster, relOptSchema, rootSchema, statement) ->
+            new RelBuilder(config.getContext(), cluster, relOptSchema));
   }
 
   /** Converts this RelBuilder to a string.
diff --git a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
index 2b35452..cb68fb9 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexExecutorTest.java
@@ -20,13 +20,10 @@ import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -71,18 +68,14 @@ public class RexExecutorTest {
   }
 
   protected void check(final Action action) throws Exception {
-    Frameworks.withPrepare(
-        new Frameworks.PrepareAction<Void>() {
-          public Void apply(RelOptCluster cluster, RelOptSchema relOptSchema,
-              SchemaPlus rootSchema, CalciteServerStatement statement) {
-            final RexBuilder rexBuilder = cluster.getRexBuilder();
-            DataContext dataContext =
-                Schemas.createDataContext(statement.getConnection(), rootSchema);
-            final RexExecutorImpl executor = new RexExecutorImpl(dataContext);
-            action.check(rexBuilder, executor);
-            return null;
-          }
-        });
+    Frameworks.withPrepare((cluster, relOptSchema, rootSchema, statement) -> {
+      final RexBuilder rexBuilder = cluster.getRexBuilder();
+      DataContext dataContext =
+          Schemas.createDataContext(statement.getConnection(), rootSchema);
+      final RexExecutorImpl executor = new RexExecutorImpl(dataContext);
+      action.check(rexBuilder, executor);
+      return null;
+    });
   }
 
   /** Tests an executor that uses variables stored in a {@link DataContext}.
@@ -217,27 +210,22 @@ public class RexExecutorTest {
       final SqlOperator operator,
       final DataContext.Variable variable,
       final Object value) {
-    Frameworks.withPrepare(new Frameworks.PrepareAction<Void>() {
-      public Void apply(final RelOptCluster cluster,
-          final RelOptSchema relOptSchema,
-          final SchemaPlus rootSchema,
-          final CalciteServerStatement statement) {
-        final RexBuilder rexBuilder = cluster.getRexBuilder();
-        final RexExecutorImpl executor =
-            new RexExecutorImpl(
-                new SingleValueDataContext(variable.camelName, value));
-        try {
-          checkConstant(value, builder -> {
-            final List<RexNode> output = new ArrayList<>();
-            executor.reduce(rexBuilder,
-                ImmutableList.of(rexBuilder.makeCall(operator)), output);
-            return output.get(0);
-          });
-        } catch (Exception e) {
-          throw TestUtil.rethrow(e);
-        }
-        return null;
+    Frameworks.withPrepare((cluster, relOptSchema, rootSchema, statement) -> {
+      final RexBuilder rexBuilder = cluster.getRexBuilder();
+      final RexExecutorImpl executor =
+          new RexExecutorImpl(
+              new SingleValueDataContext(variable.camelName, value));
+      try {
+        checkConstant(value, builder -> {
+          final List<RexNode> output = new ArrayList<>();
+          executor.reduce(rexBuilder,
+              ImmutableList.of(rexBuilder.makeCall(operator)), output);
+          return output.get(0);
+        });
+      } catch (Exception e) {
+        throw TestUtil.rethrow(e);
       }
+      return null;
     });
   }
 
diff --git a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
index 2b9b57f..7d1e900 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -16,12 +16,9 @@
  */
 package org.apache.calcite.test;
 
-import org.apache.calcite.DataContext;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RexImplicationChecker;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -34,15 +31,12 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUnknownAs;
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.util.DateString;
-import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.TimeString;
 import org.apache.calcite.util.TimestampString;
@@ -536,21 +530,11 @@ public class RexImplicationCheckerTest {
           .add("string", stringDataType)
           .build();
 
-      final Holder<RexExecutorImpl> holder = Holder.of(null);
-      Frameworks.withPrepare(
-          new Frameworks.PrepareAction<Void>() {
-            public Void apply(RelOptCluster cluster,
-                RelOptSchema relOptSchema,
-                SchemaPlus rootSchema,
-                CalciteServerStatement statement) {
-              DataContext dataContext =
-                  Schemas.createDataContext(statement.getConnection(), rootSchema);
-              holder.set(new RexExecutorImpl(dataContext));
-              return null;
-            }
-          });
-
-      executor = holder.get();
+      executor = Frameworks.withPrepare(
+          (cluster, relOptSchema, rootSchema, statement) ->
+              new RexExecutorImpl(
+                  Schemas.createDataContext(statement.getConnection(),
+                      rootSchema)));
       simplify =
           new RexSimplify(rexBuilder, RelOptPredicateList.EMPTY, executor)
               .withParanoid(true);
diff --git a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
index 29c0ec3..4bec720 100644
--- a/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/FrameworksTest.java
@@ -28,7 +28,6 @@ import org.apache.calcite.plan.ConventionTraitDef;
 import org.apache.calcite.plan.RelOptAbstractTable;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitDef;
@@ -57,7 +56,6 @@ 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.AbstractTable;
-import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SqlExplainFormat;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlNode;
@@ -173,23 +171,19 @@ public class FrameworksTest {
   }
 
   private void checkTypeSystem(final int expected, FrameworkConfig config) {
-    Frameworks.withPrepare(
-        new Frameworks.PrepareAction<Void>(config) {
-          @Override public Void apply(RelOptCluster cluster,
-              RelOptSchema relOptSchema, SchemaPlus rootSchema,
-              CalciteServerStatement statement) {
-            final RelDataType type =
-                cluster.getTypeFactory()
-                    .createSqlType(SqlTypeName.DECIMAL, 30, 2);
-            final RexLiteral literal =
-                cluster.getRexBuilder().makeExactLiteral(BigDecimal.ONE, type);
-            final RexNode call =
-                cluster.getRexBuilder().makeCall(SqlStdOperatorTable.PLUS,
-                    literal,
-                    literal);
-            assertEquals(expected, call.getType().getPrecision());
-            return null;
-          }
+    Frameworks.withPrepare(config,
+        (cluster, relOptSchema, rootSchema, statement) -> {
+          final RelDataType type =
+              cluster.getTypeFactory()
+                  .createSqlType(SqlTypeName.DECIMAL, 30, 2);
+          final RexLiteral literal =
+              cluster.getRexBuilder().makeExactLiteral(BigDecimal.ONE, type);
+          final RexNode call =
+              cluster.getRexBuilder().makeCall(SqlStdOperatorTable.PLUS,
+                  literal,
+                  literal);
+          assertEquals(expected, call.getType().getPrecision());
+          return null;
         });
   }