You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/03/23 14:36:40 UTC

[GitHub] [flink] twalthr commented on a change in pull request #19205: [hotfix][table-planner] Cleanup code around TableConfig/ReadableConfig

twalthr commented on a change in pull request #19205:
URL: https://github.com/apache/flink/pull/19205#discussion_r833339814



##########
File path: flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala
##########
@@ -520,7 +499,6 @@ class AggCallSelectivityEstimatorTest {
     // count(amount), count(price) group by name
     val agg1 = createAggregate(Array(name_idx),
       Seq((SqlStdOperatorTable.COUNT, amount_idx), (SqlStdOperatorTable.COUNT, price_idx)))
-    val se = new SelectivityEstimator(agg1, mq)

Review comment:
       are you sure this is not necessary?

##########
File path: flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
##########
@@ -82,27 +78,18 @@ import scala.collection.JavaConversions._
 class FlinkRelMdHandlerTestBase {
 
   val tableConfig = TableConfig.getDefault()
-  val rootSchema: SchemaPlus = MetadataTestUtil.initRootSchema()
-
-  val catalogManager: CatalogManager = CatalogManagerMocks.createEmptyCatalogManager()
-  val moduleManager = new ModuleManager
 
   // TODO batch RelNode and stream RelNode should have different PlannerContext
   //  and RelOptCluster due to they have different trait definitions.
-  val plannerContext: PlannerContext =
-  new PlannerContext(
-    false,
-    tableConfig,
-    moduleManager,
-    new FunctionCatalog(tableConfig, catalogManager, moduleManager),
-    catalogManager,
-    CalciteSchema.from(rootSchema),
-    util.Arrays.asList(
-      ConventionTraitDef.INSTANCE,
-      FlinkRelDistributionTraitDef.INSTANCE,
-      RelCollationTraitDef.INSTANCE
-    )
-  )
+  val plannerContext: PlannerContext = new PlannerMocks.Builder()
+      .withTableConfig(tableConfig)
+      .withRootSchema(CalciteSchema.from(MetadataTestUtil.initRootSchema()))
+      .withTraitDefs(util.Arrays.asList(

Review comment:
       not necessary?

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java
##########
@@ -103,6 +103,7 @@
     private final List<RelTraitDef> traitDefs;
     private final FrameworkConfig frameworkConfig;
 
+    /** Tests should use {@code PlannerMocks.Builder} or {@code PlannerMocks.create} instead. */

Review comment:
       drop this? a bit unusual to have production code giving advise for tests

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java
##########
@@ -88,7 +89,8 @@ private FlinkRelBuilder(Context context, RelOptCluster cluster, RelOptSchema rel
         super(context, cluster, relOptSchema);
 
         this.toRelNodeConverter =
-                new QueryOperationConverter(this, context.unwrap(FlinkContext.class).isBatchMode());
+                new QueryOperationConverter(
+                        this, ShortcutUtils.unwrapContext(context).isBatchMode());

Review comment:
       nit: use static imports everywhere to make the code more readable

##########
File path: flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
##########
@@ -82,27 +78,18 @@ import scala.collection.JavaConversions._
 class FlinkRelMdHandlerTestBase {
 
   val tableConfig = TableConfig.getDefault()
-  val rootSchema: SchemaPlus = MetadataTestUtil.initRootSchema()
-
-  val catalogManager: CatalogManager = CatalogManagerMocks.createEmptyCatalogManager()
-  val moduleManager = new ModuleManager
 
   // TODO batch RelNode and stream RelNode should have different PlannerContext
   //  and RelOptCluster due to they have different trait definitions.
-  val plannerContext: PlannerContext =
-  new PlannerContext(
-    false,
-    tableConfig,
-    moduleManager,
-    new FunctionCatalog(tableConfig, catalogManager, moduleManager),
-    catalogManager,
-    CalciteSchema.from(rootSchema),
-    util.Arrays.asList(
-      ConventionTraitDef.INSTANCE,
-      FlinkRelDistributionTraitDef.INSTANCE,
-      RelCollationTraitDef.INSTANCE
-    )
-  )
+  val plannerContext: PlannerContext = new PlannerMocks.Builder()

Review comment:
       nit: can we do `PlannerMocks.newBuilder()` this is a bit more common in the code base

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala
##########
@@ -75,25 +76,22 @@ class LegacyCatalogSourceTable[T](
 
   override def toRel(context: RelOptTable.ToRelContext): RelNode = {
     val cluster = context.getCluster
-    val flinkContext = cluster
-      .getPlanner
-      .getContext
-      .unwrap(classOf[FlinkContext])
+    val flinkContext = ShortcutUtils.unwrapContext(cluster)

Review comment:
       is `flinkContext` used again? if not we can remove it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org