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



##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Provides a flexible testing harness for table factories.
+ *
+ * <p>This testing harness allows writing custom sources and sinks which can 
be directly
+ * instantiated from the test. This avoids having to implement a factory, and 
enables using the
+ * {@link SharedObjects} rule to get direct access to the underlying 
source/sink from the test.
+ *
+ * <p>Note that the underlying source/sink must be {@link Serializable}. It is 
recommended to extend
+ * from {@link ScanSourceBase}, {@link LookupSourceBase}, or {@link SinkBase} 
which provide default
+ * implementations for most methods as well as some convenience methods.
+ *
+ * <p>The harness provides a {@link Factory}. You can register a source / sink 
through configuration
+ * by passing a base64-encoded serialization. The harness provides convenience 
methods to make this
+ * process as simple as possible.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * public class CustomSourceTest {
+ *     {@literal @}Rule public SharedObjects sharedObjects = 
SharedObjects.create();
+ *
+ *     {@literal @}Test
+ *     public void test() {
+ *         SharedReference<List<Long>> appliedLimits = sharedObjects.add(new 
ArrayList<>());
+ *
+ *         Schema schema = Schema.newBuilder().build();

Review comment:
       nit: use the new `Schema.derived()`

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Provides a flexible testing harness for table factories.
+ *
+ * <p>This testing harness allows writing custom sources and sinks which can 
be directly
+ * instantiated from the test. This avoids having to implement a factory, and 
enables using the
+ * {@link SharedObjects} rule to get direct access to the underlying 
source/sink from the test.
+ *
+ * <p>Note that the underlying source/sink must be {@link Serializable}. It is 
recommended to extend
+ * from {@link ScanSourceBase}, {@link LookupSourceBase}, or {@link SinkBase} 
which provide default
+ * implementations for most methods as well as some convenience methods.
+ *
+ * <p>The harness provides a {@link Factory}. You can register a source / sink 
through configuration
+ * by passing a base64-encoded serialization. The harness provides convenience 
methods to make this
+ * process as simple as possible.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * public class CustomSourceTest {
+ *     {@literal @}Rule public SharedObjects sharedObjects = 
SharedObjects.create();
+ *
+ *     {@literal @}Test
+ *     public void test() {
+ *         SharedReference<List<Long>> appliedLimits = sharedObjects.add(new 
ArrayList<>());
+ *
+ *         Schema schema = Schema.newBuilder().build();
+ *         TableDescriptor sourceDescriptor = 
TableFactoryHarness.forSource(schema,
+ *             new CustomSource(appliedLimits));
+ *
+ *         tEnv.createTable("T", sourceDescriptor);
+ *         tEnv.explainSql("SELECT * FROM T LIMIT 42");
+ *
+ *         assertEquals(1, appliedLimits.get().size());
+ *         assertEquals((Long) 42L, appliedLimits.get().get(0));
+ *     }
+ *
+ *     private static class CustomSource extends ScanSourceBase implements 
SupportsLimitPushDown {
+ *         private final SharedReference<List<Long>> appliedLimits;
+ *
+ *         CustomSource(SharedReference<List<Long>> appliedLimits) {
+ *             this.appliedLimits = appliedLimits;
+ *         }
+ *
+ *         {@literal @}Override
+ *         public void applyLimit(long limit) {
+ *             appliedLimits.get().add(limit);
+ *         }
+ *     }
+ * }
+ * }</pre>
+ */
+public class TableFactoryHarness {
+
+    /** Factory identifier for {@link Factory}. */
+    public static final String IDENTIFIER = "harness";
+
+    public static final ConfigOption<String> SOURCE =
+            ConfigOptions.key("source")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of 
DynamicTableSource (Base64-encoded)");
+
+    public static final ConfigOption<String> SINK =
+            ConfigOptions.key("sink")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of DynamicTableSink 
(Base64-encoded)");
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param source}. */
+    public static TableDescriptor forSource(Schema schema, SourceBase source) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SOURCE, source.serialize())
+                .build();
+    }
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param sink}. */
+    public static TableDescriptor forSink(Schema schema, SinkBase sink) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SINK, sink.serialize())
+                .build();
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Harness factory for creating sources / sinks from base64-encoded 
serialized strings. */
+    public static class Factory implements DynamicTableSourceFactory, 
DynamicTableSinkFactory {
+
+        @Override
+        public String factoryIdentifier() {
+            return IDENTIFIER;
+        }
+
+        @Override
+        public Set<ConfigOption<?>> requiredOptions() {
+            return new HashSet<>();
+        }
+
+        @Override
+        public Set<ConfigOption<?>> optionalOptions() {
+            Set<ConfigOption<?>> options = new HashSet<>();
+            options.add(SOURCE);
+            options.add(SINK);
+            return options;
+        }
+
+        @Override
+        public DynamicTableSource createDynamicTableSource(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSource source =
+                    deserializeSourceSink(factoryHelper.getOptions(), SOURCE);
+            if (source instanceof SourceBase) {
+                ((SourceBase) source).factoryContext = context;
+            }
+
+            return source;
+        }
+
+        @Override
+        public DynamicTableSink createDynamicTableSink(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSink sink = 
deserializeSourceSink(factoryHelper.getOptions(), SINK);
+            if (sink instanceof SinkBase) {
+                ((SinkBase) sink).factoryContext = context;
+            }
+
+            return sink;
+        }
+
+        private <T> T deserializeSourceSink(
+                ReadableConfig options, ConfigOption<String> configOption) {
+            final String serializedValue =
+                    options.getOptional(configOption)
+                            .orElseThrow(
+                                    () ->
+                                            new ValidationException(
+                                                    String.format(
+                                                            "Missing option 
'%s'.",
+                                                            
configOption.key())));
+
+            try {
+                return InstantiationUtil.deserializeObject(
+                        Base64.getDecoder().decode(serializedValue),
+                        Thread.currentThread().getContextClassLoader());
+            } catch (ClassNotFoundException | IOException e) {
+                throw new ValidationException(
+                        "Serialized source/sink could not be deserialized.", 
e);
+            }
+        }
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /**
+     * Serializes a source / sink into a base64-encoded string which can be 
used by {@link Factory}.
+     *
+     * <p>If your source / sink extends from {@link ScanSourceBase}, {@link 
LookupSourceBase}, or
+     * {@link SinkBase}, you can use {@link SourceBase#serialize()} / {@link 
SinkBase#serialize()}
+     * instead, or use {@link #forSource(Schema, SourceBase)} / {@link 
#forSink(Schema, SinkBase)}.
+     */
+    public static String serializeImplementation(Object obj) {

Review comment:
       the serialize/deserialize part doesn't look very nice. it could also 
slow down the test, wouldn't it be possible to somehow leverage the 
`SharedObjects` infrastructure and only pass around `sharedObjectsId`?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Provides a flexible testing harness for table factories.
+ *
+ * <p>This testing harness allows writing custom sources and sinks which can 
be directly
+ * instantiated from the test. This avoids having to implement a factory, and 
enables using the
+ * {@link SharedObjects} rule to get direct access to the underlying 
source/sink from the test.
+ *
+ * <p>Note that the underlying source/sink must be {@link Serializable}. It is 
recommended to extend
+ * from {@link ScanSourceBase}, {@link LookupSourceBase}, or {@link SinkBase} 
which provide default
+ * implementations for most methods as well as some convenience methods.
+ *
+ * <p>The harness provides a {@link Factory}. You can register a source / sink 
through configuration
+ * by passing a base64-encoded serialization. The harness provides convenience 
methods to make this
+ * process as simple as possible.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * public class CustomSourceTest {
+ *     {@literal @}Rule public SharedObjects sharedObjects = 
SharedObjects.create();
+ *
+ *     {@literal @}Test
+ *     public void test() {
+ *         SharedReference<List<Long>> appliedLimits = sharedObjects.add(new 
ArrayList<>());
+ *
+ *         Schema schema = Schema.newBuilder().build();
+ *         TableDescriptor sourceDescriptor = 
TableFactoryHarness.forSource(schema,
+ *             new CustomSource(appliedLimits));
+ *
+ *         tEnv.createTable("T", sourceDescriptor);
+ *         tEnv.explainSql("SELECT * FROM T LIMIT 42");
+ *
+ *         assertEquals(1, appliedLimits.get().size());
+ *         assertEquals((Long) 42L, appliedLimits.get().get(0));
+ *     }
+ *
+ *     private static class CustomSource extends ScanSourceBase implements 
SupportsLimitPushDown {
+ *         private final SharedReference<List<Long>> appliedLimits;
+ *
+ *         CustomSource(SharedReference<List<Long>> appliedLimits) {
+ *             this.appliedLimits = appliedLimits;
+ *         }
+ *
+ *         {@literal @}Override
+ *         public void applyLimit(long limit) {
+ *             appliedLimits.get().add(limit);
+ *         }
+ *     }
+ * }
+ * }</pre>
+ */
+public class TableFactoryHarness {
+
+    /** Factory identifier for {@link Factory}. */
+    public static final String IDENTIFIER = "harness";
+
+    public static final ConfigOption<String> SOURCE =
+            ConfigOptions.key("source")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of 
DynamicTableSource (Base64-encoded)");
+
+    public static final ConfigOption<String> SINK =
+            ConfigOptions.key("sink")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of DynamicTableSink 
(Base64-encoded)");
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param source}. */
+    public static TableDescriptor forSource(Schema schema, SourceBase source) {

Review comment:
       can we add a variant that does not require `SourceBase` instance? This 
could be useful for basic plan tests with no abilities implemented.

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushFilterPastChangelogNormalizeRule.java
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.physical.stream;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.plan.utils.RexNodeExtractor.extractRefInputFields;
+
+/**
+ * Pushes primary key filters through a {@link 
StreamPhysicalChangelogNormalize ChangelogNormalize}
+ * operator to reduce its state size.
+ *
+ * <p>This rule looks for Calc → ChangelogNormalize where the {@link 
StreamPhysicalCalc Calc}
+ * contains a filter condition. The condition is transformed into CNF and then 
each conjunction is
+ * tested for whether it affects only primary key columns. If such conditions 
exist, they are moved
+ * into a new, separate Calc and pushed through the ChangelogNormalize 
operator. ChangelogNormalize
+ * keeps state for every unique key it encounters, thus pushing filters on the 
primary key in front
+ * of it helps reduce the size of its state.
+ *
+ * <p>Note that pushing primary key filters is safe to do, but pushing any 
other filters would lead
+ * to incorrect results.

Review comment:
       nit: `potentially incorrect results` it depends on the query

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushFilterPastChangelogNormalizeRule.java
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.physical.stream;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.plan.utils.RexNodeExtractor.extractRefInputFields;
+
+/**
+ * Pushes primary key filters through a {@link 
StreamPhysicalChangelogNormalize ChangelogNormalize}
+ * operator to reduce its state size.
+ *
+ * <p>This rule looks for Calc → ChangelogNormalize where the {@link 
StreamPhysicalCalc Calc}
+ * contains a filter condition. The condition is transformed into CNF and then 
each conjunction is
+ * tested for whether it affects only primary key columns. If such conditions 
exist, they are moved
+ * into a new, separate Calc and pushed through the ChangelogNormalize 
operator. ChangelogNormalize
+ * keeps state for every unique key it encounters, thus pushing filters on the 
primary key in front
+ * of it helps reduce the size of its state.
+ *
+ * <p>Note that pushing primary key filters is safe to do, but pushing any 
other filters would lead
+ * to incorrect results.
+ */
+@Internal
+public class PushFilterPastChangelogNormalizeRule
+        extends RelRule<PushFilterPastChangelogNormalizeRule.Config> {
+
+    public static final RelOptRule INSTANCE =
+            
Config.EMPTY.as(Config.class).onFilterWithChangelogNormalize().toRule();
+
+    public PushFilterPastChangelogNormalizeRule(Config config) {
+        super(config);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final StreamPhysicalCalc calc = call.rel(0);
+        final StreamPhysicalChangelogNormalize changelogNormalize = 
call.rel(1);
+
+        final RexProgram program = calc.getProgram();
+        final RexNode condition =
+                RexUtil.toCnf(
+                        call.builder().getRexBuilder(),
+                        program.expandLocalRef(program.getCondition()));
+
+        final Set<Integer> primaryKeyIndices =
+                
IntStream.of(changelogNormalize.uniqueKeys()).boxed().collect(Collectors.toSet());
+
+        // Determine which filters can be pushed (= involve only primary key 
columns)
+        final List<RexNode> primaryKeyPredicates = new ArrayList<>();
+        final List<RexNode> otherPredicates = new ArrayList<>();
+        partitionPrimaryKeyPredicates(
+                RelOptUtil.conjunctions(condition),
+                primaryKeyIndices,
+                primaryKeyPredicates,
+                otherPredicates);
+
+        // Construct a new ChangelogNormalize which has primary key filters 
pushed into it
+        final StreamPhysicalChangelogNormalize newChangelogNormalize =
+                pushFiltersThroughChangeNormalize(call, primaryKeyPredicates);
+
+        // Retain only filters which haven't been pushed
+        transformWithRemainingPredicates(call, newChangelogNormalize, 
otherPredicates);
+    }
+
+    /**
+     * Separates the given {@param predicates} into filters which affect only 
the primary key and
+     * anything else.
+     */
+    private void partitionPrimaryKeyPredicates(
+            List<RexNode> predicates,
+            Set<Integer> primaryKeyIndices,
+            List<RexNode> primaryKeyPredicates,
+            List<RexNode> remainingPredicates) {
+        for (RexNode predicate : predicates) {
+            int[] inputRefs = 
extractRefInputFields(Collections.singletonList(predicate));
+            if 
(Arrays.stream(inputRefs).allMatch(primaryKeyIndices::contains)) {
+                primaryKeyPredicates.add(predicate);
+            } else {
+                remainingPredicates.add(predicate);
+            }
+        }
+    }
+
+    /** Pushes {@param primaryKeyPredicates} into the {@link 
StreamPhysicalChangelogNormalize}. */
+    private StreamPhysicalChangelogNormalize pushFiltersThroughChangeNormalize(

Review comment:
       nit: `pushFiltersThroughChangelogNormalize`

##########
File path: 
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/PushFilterPastChangelogNormalizeRuleTest.xml
##########
@@ -30,10 +30,11 @@ LogicalAggregate(group=[{0}], sum=[SUM($0)])
     </Resource>
     <Resource name="optimized rel plan">
       <![CDATA[
-Calc(select=[f1, f1 AS sum], where=[AND(<(f1, 10), OR(>(f1, 3), IS NULL(f2)))])
+Calc(select=[f1, f1 AS sum], where=[OR(>(f1, 3), IS NULL(f2))])
 +- ChangelogNormalize(key=[f1])
-   +- Exchange(distribution=[hash[f1]])
-      +- TableSourceScan(table=[[default_catalog, default_database, T]], 
fields=[f0, f1, f2])
+   +- Calc(select=[f0, f1, f2], where=[<(f1, 10)])
+      +- Exchange(distribution=[hash[f1]])

Review comment:
       sorry, if I was not clear enough. but of course we can also push it 
through the exchange that will (always?) be in front of the 
`ChangelogNormalize`. Shuffling is also expensive due to network IO, the more 
filtering here, the better.

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Provides a flexible testing harness for table factories.
+ *
+ * <p>This testing harness allows writing custom sources and sinks which can 
be directly
+ * instantiated from the test. This avoids having to implement a factory, and 
enables using the
+ * {@link SharedObjects} rule to get direct access to the underlying 
source/sink from the test.
+ *
+ * <p>Note that the underlying source/sink must be {@link Serializable}. It is 
recommended to extend
+ * from {@link ScanSourceBase}, {@link LookupSourceBase}, or {@link SinkBase} 
which provide default
+ * implementations for most methods as well as some convenience methods.
+ *
+ * <p>The harness provides a {@link Factory}. You can register a source / sink 
through configuration
+ * by passing a base64-encoded serialization. The harness provides convenience 
methods to make this
+ * process as simple as possible.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * public class CustomSourceTest {
+ *     {@literal @}Rule public SharedObjects sharedObjects = 
SharedObjects.create();
+ *
+ *     {@literal @}Test
+ *     public void test() {
+ *         SharedReference<List<Long>> appliedLimits = sharedObjects.add(new 
ArrayList<>());
+ *
+ *         Schema schema = Schema.newBuilder().build();
+ *         TableDescriptor sourceDescriptor = 
TableFactoryHarness.forSource(schema,
+ *             new CustomSource(appliedLimits));
+ *
+ *         tEnv.createTable("T", sourceDescriptor);
+ *         tEnv.explainSql("SELECT * FROM T LIMIT 42");
+ *
+ *         assertEquals(1, appliedLimits.get().size());
+ *         assertEquals((Long) 42L, appliedLimits.get().get(0));
+ *     }
+ *
+ *     private static class CustomSource extends ScanSourceBase implements 
SupportsLimitPushDown {
+ *         private final SharedReference<List<Long>> appliedLimits;
+ *
+ *         CustomSource(SharedReference<List<Long>> appliedLimits) {
+ *             this.appliedLimits = appliedLimits;
+ *         }
+ *
+ *         {@literal @}Override
+ *         public void applyLimit(long limit) {
+ *             appliedLimits.get().add(limit);
+ *         }
+ *     }
+ * }
+ * }</pre>
+ */
+public class TableFactoryHarness {
+
+    /** Factory identifier for {@link Factory}. */
+    public static final String IDENTIFIER = "harness";
+
+    public static final ConfigOption<String> SOURCE =
+            ConfigOptions.key("source")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of 
DynamicTableSource (Base64-encoded)");
+
+    public static final ConfigOption<String> SINK =
+            ConfigOptions.key("sink")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of DynamicTableSink 
(Base64-encoded)");
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param source}. */
+    public static TableDescriptor forSource(Schema schema, SourceBase source) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SOURCE, source.serialize())
+                .build();
+    }
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param sink}. */
+    public static TableDescriptor forSink(Schema schema, SinkBase sink) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SINK, sink.serialize())
+                .build();
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Harness factory for creating sources / sinks from base64-encoded 
serialized strings. */
+    public static class Factory implements DynamicTableSourceFactory, 
DynamicTableSinkFactory {
+
+        @Override
+        public String factoryIdentifier() {
+            return IDENTIFIER;
+        }
+
+        @Override
+        public Set<ConfigOption<?>> requiredOptions() {
+            return new HashSet<>();
+        }
+
+        @Override
+        public Set<ConfigOption<?>> optionalOptions() {
+            Set<ConfigOption<?>> options = new HashSet<>();
+            options.add(SOURCE);
+            options.add(SINK);
+            return options;
+        }
+
+        @Override
+        public DynamicTableSource createDynamicTableSource(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSource source =
+                    deserializeSourceSink(factoryHelper.getOptions(), SOURCE);
+            if (source instanceof SourceBase) {
+                ((SourceBase) source).factoryContext = context;
+            }
+
+            return source;
+        }
+
+        @Override
+        public DynamicTableSink createDynamicTableSink(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSink sink = 
deserializeSourceSink(factoryHelper.getOptions(), SINK);
+            if (sink instanceof SinkBase) {
+                ((SinkBase) sink).factoryContext = context;
+            }
+
+            return sink;
+        }
+
+        private <T> T deserializeSourceSink(
+                ReadableConfig options, ConfigOption<String> configOption) {
+            final String serializedValue =
+                    options.getOptional(configOption)
+                            .orElseThrow(
+                                    () ->
+                                            new ValidationException(
+                                                    String.format(
+                                                            "Missing option 
'%s'.",
+                                                            
configOption.key())));
+
+            try {
+                return InstantiationUtil.deserializeObject(
+                        Base64.getDecoder().decode(serializedValue),
+                        Thread.currentThread().getContextClassLoader());
+            } catch (ClassNotFoundException | IOException e) {
+                throw new ValidationException(
+                        "Serialized source/sink could not be deserialized.", 
e);
+            }
+        }
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /**
+     * Serializes a source / sink into a base64-encoded string which can be 
used by {@link Factory}.
+     *
+     * <p>If your source / sink extends from {@link ScanSourceBase}, {@link 
LookupSourceBase}, or
+     * {@link SinkBase}, you can use {@link SourceBase#serialize()} / {@link 
SinkBase#serialize()}
+     * instead, or use {@link #forSource(Schema, SourceBase)} / {@link 
#forSink(Schema, SinkBase)}.
+     */
+    public static String serializeImplementation(Object obj) {
+        try {
+            return 
Base64.getEncoder().encodeToString(InstantiationUtil.serializeObject(obj));
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+    }
+
+    /** Creates a {@link ScanRuntimeProvider} which produces nothing. */
+    public static ScanRuntimeProvider createEmptyScanProvider() {
+        return SourceFunctionProvider.of(
+                new SourceFunction<RowData>() {
+                    @Override
+                    public void run(SourceContext<RowData> ctx) {}
+
+                    @Override
+                    public void cancel() {}
+                },
+                true);

Review comment:
       it would be useful to make this configurable, maybe even via the 
`TableFactoryHarness.forBoundedSource`, 
``TableFactoryHarness.forUnboundedSource``

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushFilterPastChangelogNormalizeRule.java
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.physical.stream;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.plan.utils.RexNodeExtractor.extractRefInputFields;
+
+/**
+ * Pushes primary key filters through a {@link 
StreamPhysicalChangelogNormalize ChangelogNormalize}
+ * operator to reduce its state size.
+ *
+ * <p>This rule looks for Calc → ChangelogNormalize where the {@link 
StreamPhysicalCalc Calc}
+ * contains a filter condition. The condition is transformed into CNF and then 
each conjunction is
+ * tested for whether it affects only primary key columns. If such conditions 
exist, they are moved
+ * into a new, separate Calc and pushed through the ChangelogNormalize 
operator. ChangelogNormalize
+ * keeps state for every unique key it encounters, thus pushing filters on the 
primary key in front
+ * of it helps reduce the size of its state.
+ *
+ * <p>Note that pushing primary key filters is safe to do, but pushing any 
other filters would lead
+ * to incorrect results.
+ */
+@Internal
+public class PushFilterPastChangelogNormalizeRule
+        extends RelRule<PushFilterPastChangelogNormalizeRule.Config> {
+
+    public static final RelOptRule INSTANCE =
+            
Config.EMPTY.as(Config.class).onFilterWithChangelogNormalize().toRule();
+
+    public PushFilterPastChangelogNormalizeRule(Config config) {
+        super(config);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final StreamPhysicalCalc calc = call.rel(0);
+        final StreamPhysicalChangelogNormalize changelogNormalize = 
call.rel(1);
+
+        final RexProgram program = calc.getProgram();
+        final RexNode condition =
+                RexUtil.toCnf(
+                        call.builder().getRexBuilder(),
+                        program.expandLocalRef(program.getCondition()));
+
+        final Set<Integer> primaryKeyIndices =
+                
IntStream.of(changelogNormalize.uniqueKeys()).boxed().collect(Collectors.toSet());
+
+        // Determine which filters can be pushed (= involve only primary key 
columns)
+        final List<RexNode> primaryKeyPredicates = new ArrayList<>();
+        final List<RexNode> otherPredicates = new ArrayList<>();
+        partitionPrimaryKeyPredicates(
+                RelOptUtil.conjunctions(condition),
+                primaryKeyIndices,
+                primaryKeyPredicates,
+                otherPredicates);
+
+        // Construct a new ChangelogNormalize which has primary key filters 
pushed into it
+        final StreamPhysicalChangelogNormalize newChangelogNormalize =
+                pushFiltersThroughChangeNormalize(call, primaryKeyPredicates);
+
+        // Retain only filters which haven't been pushed
+        transformWithRemainingPredicates(call, newChangelogNormalize, 
otherPredicates);
+    }
+
+    /**
+     * Separates the given {@param predicates} into filters which affect only 
the primary key and
+     * anything else.
+     */
+    private void partitionPrimaryKeyPredicates(
+            List<RexNode> predicates,
+            Set<Integer> primaryKeyIndices,
+            List<RexNode> primaryKeyPredicates,
+            List<RexNode> remainingPredicates) {
+        for (RexNode predicate : predicates) {
+            int[] inputRefs = 
extractRefInputFields(Collections.singletonList(predicate));
+            if 
(Arrays.stream(inputRefs).allMatch(primaryKeyIndices::contains)) {

Review comment:
       I'm just mentioning here if it was a conscious decision to support use 
cases such as:
   primary key with `(a,b,c)` with a predicate `a=1 && b=42` would be pushed 
through?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/PushFilterPastChangelogNormalizeRule.java
##########
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.physical.stream;
+
+import org.apache.flink.annotation.Internal;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc;
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.flink.table.planner.plan.utils.RexNodeExtractor.extractRefInputFields;
+
+/**
+ * Pushes primary key filters through a {@link 
StreamPhysicalChangelogNormalize ChangelogNormalize}
+ * operator to reduce its state size.
+ *
+ * <p>This rule looks for Calc → ChangelogNormalize where the {@link 
StreamPhysicalCalc Calc}
+ * contains a filter condition. The condition is transformed into CNF and then 
each conjunction is
+ * tested for whether it affects only primary key columns. If such conditions 
exist, they are moved
+ * into a new, separate Calc and pushed through the ChangelogNormalize 
operator. ChangelogNormalize
+ * keeps state for every unique key it encounters, thus pushing filters on the 
primary key in front
+ * of it helps reduce the size of its state.
+ *
+ * <p>Note that pushing primary key filters is safe to do, but pushing any 
other filters would lead
+ * to incorrect results.
+ */
+@Internal
+public class PushFilterPastChangelogNormalizeRule
+        extends RelRule<PushFilterPastChangelogNormalizeRule.Config> {
+
+    public static final RelOptRule INSTANCE =
+            
Config.EMPTY.as(Config.class).onFilterWithChangelogNormalize().toRule();
+
+    public PushFilterPastChangelogNormalizeRule(Config config) {
+        super(config);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+        final StreamPhysicalCalc calc = call.rel(0);
+        final StreamPhysicalChangelogNormalize changelogNormalize = 
call.rel(1);
+
+        final RexProgram program = calc.getProgram();
+        final RexNode condition =
+                RexUtil.toCnf(
+                        call.builder().getRexBuilder(),
+                        program.expandLocalRef(program.getCondition()));
+
+        final Set<Integer> primaryKeyIndices =
+                
IntStream.of(changelogNormalize.uniqueKeys()).boxed().collect(Collectors.toSet());
+
+        // Determine which filters can be pushed (= involve only primary key 
columns)
+        final List<RexNode> primaryKeyPredicates = new ArrayList<>();
+        final List<RexNode> otherPredicates = new ArrayList<>();
+        partitionPrimaryKeyPredicates(
+                RelOptUtil.conjunctions(condition),
+                primaryKeyIndices,
+                primaryKeyPredicates,
+                otherPredicates);
+
+        // Construct a new ChangelogNormalize which has primary key filters 
pushed into it
+        final StreamPhysicalChangelogNormalize newChangelogNormalize =
+                pushFiltersThroughChangeNormalize(call, primaryKeyPredicates);
+
+        // Retain only filters which haven't been pushed
+        transformWithRemainingPredicates(call, newChangelogNormalize, 
otherPredicates);
+    }
+
+    /**
+     * Separates the given {@param predicates} into filters which affect only 
the primary key and
+     * anything else.
+     */
+    private void partitionPrimaryKeyPredicates(
+            List<RexNode> predicates,
+            Set<Integer> primaryKeyIndices,
+            List<RexNode> primaryKeyPredicates,
+            List<RexNode> remainingPredicates) {
+        for (RexNode predicate : predicates) {
+            int[] inputRefs = 
extractRefInputFields(Collections.singletonList(predicate));
+            if 
(Arrays.stream(inputRefs).allMatch(primaryKeyIndices::contains)) {
+                primaryKeyPredicates.add(predicate);
+            } else {
+                remainingPredicates.add(predicate);
+            }
+        }
+    }
+
+    /** Pushes {@param primaryKeyPredicates} into the {@link 
StreamPhysicalChangelogNormalize}. */
+    private StreamPhysicalChangelogNormalize pushFiltersThroughChangeNormalize(
+            RelOptRuleCall call, List<RexNode> primaryKeyPredicates) {
+        final StreamPhysicalChangelogNormalize changelogNormalize = 
call.rel(1);
+
+        if (primaryKeyPredicates.isEmpty()) {
+            // There are no filters which can be pushed, so just return the 
existing node.
+            return changelogNormalize;
+        }
+
+        final StreamPhysicalCalc pushedFiltersCalc =
+                projectIdentityWithConditions(
+                        call.builder(), changelogNormalize.getInput(), 
primaryKeyPredicates);
+
+        return (StreamPhysicalChangelogNormalize)
+                changelogNormalize.copy(
+                        changelogNormalize.getTraitSet(),
+                        Collections.singletonList(pushedFiltersCalc));
+    }
+
+    /**
+     * Returns a {@link StreamPhysicalCalc} with the given {@param conditions} 
and an identity
+     * projection.
+     */
+    private StreamPhysicalCalc projectIdentityWithConditions(
+            RelBuilder relBuilder, RelNode newInput, List<RexNode> conditions) 
{
+
+        final RexProgramBuilder programBuilder =
+                new RexProgramBuilder(newInput.getRowType(), 
relBuilder.getRexBuilder());
+        programBuilder.addIdentity();
+
+        final RexNode condition = relBuilder.and(conditions);
+        if (!condition.isAlwaysTrue()) {
+            programBuilder.addCondition(condition);
+        }
+
+        final RexProgram newProgram = programBuilder.getProgram();
+        return new StreamPhysicalCalc(
+                newInput.getCluster(),
+                newInput.getTraitSet(),
+                newInput,
+                newProgram,
+                newProgram.getOutputRowType());
+    }
+
+    /**
+     * Returns a {@link StreamPhysicalCalc} which is a copy of {@param calc}, 
but with the
+     * projections applied from {@param projectFromCalc}.
+     */
+    private StreamPhysicalCalc projectWith(
+            RelBuilder relBuilder, StreamPhysicalCalc projectFromCalc, 
StreamPhysicalCalc calc) {
+        final RexProgramBuilder programBuilder =
+                new RexProgramBuilder(calc.getRowType(), 
relBuilder.getRexBuilder());
+        if (calc.getProgram().getCondition() != null) {
+            programBuilder.addCondition(
+                    
calc.getProgram().expandLocalRef(calc.getProgram().getCondition()));
+        }
+
+        for (Pair<RexLocalRef, String> projectRef :
+                projectFromCalc.getProgram().getNamedProjects()) {
+            final RexNode project = 
projectFromCalc.getProgram().expandLocalRef(projectRef.left);
+            programBuilder.addProject(project, projectRef.right);
+        }
+
+        final RexProgram newProgram = programBuilder.getProgram();

Review comment:
       will the resulting program have local refs again? we are not using them 
yet but just for the future, we should restore a compact representation after 
applying this rule

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import 
org.apache.flink.table.connector.sink.DynamicTableSink.SinkRuntimeProvider;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Provides a flexible testing harness for table factories.
+ *
+ * <p>This testing harness allows writing custom sources and sinks which can 
be directly
+ * instantiated from the test. This avoids having to implement a factory, and 
enables using the
+ * {@link SharedObjects} rule to get direct access to the underlying 
source/sink from the test.
+ *
+ * <p>Note that the underlying source/sink must be {@link Serializable}. It is 
recommended to extend
+ * from {@link ScanSourceBase}, {@link LookupSourceBase}, or {@link SinkBase} 
which provide default
+ * implementations for most methods as well as some convenience methods.
+ *
+ * <p>The harness provides a {@link Factory}. You can register a source / sink 
through configuration
+ * by passing a base64-encoded serialization. The harness provides convenience 
methods to make this
+ * process as simple as possible.
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * public class CustomSourceTest {
+ *     {@literal @}Rule public SharedObjects sharedObjects = 
SharedObjects.create();
+ *
+ *     {@literal @}Test
+ *     public void test() {
+ *         SharedReference<List<Long>> appliedLimits = sharedObjects.add(new 
ArrayList<>());
+ *
+ *         Schema schema = Schema.newBuilder().build();
+ *         TableDescriptor sourceDescriptor = 
TableFactoryHarness.forSource(schema,
+ *             new CustomSource(appliedLimits));
+ *
+ *         tEnv.createTable("T", sourceDescriptor);
+ *         tEnv.explainSql("SELECT * FROM T LIMIT 42");
+ *
+ *         assertEquals(1, appliedLimits.get().size());
+ *         assertEquals((Long) 42L, appliedLimits.get().get(0));
+ *     }
+ *
+ *     private static class CustomSource extends ScanSourceBase implements 
SupportsLimitPushDown {
+ *         private final SharedReference<List<Long>> appliedLimits;
+ *
+ *         CustomSource(SharedReference<List<Long>> appliedLimits) {
+ *             this.appliedLimits = appliedLimits;
+ *         }
+ *
+ *         {@literal @}Override
+ *         public void applyLimit(long limit) {
+ *             appliedLimits.get().add(limit);
+ *         }
+ *     }
+ * }
+ * }</pre>
+ */
+public class TableFactoryHarness {
+
+    /** Factory identifier for {@link Factory}. */
+    public static final String IDENTIFIER = "harness";
+
+    public static final ConfigOption<String> SOURCE =
+            ConfigOptions.key("source")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of 
DynamicTableSource (Base64-encoded)");
+
+    public static final ConfigOption<String> SINK =
+            ConfigOptions.key("sink")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Serialized instance of DynamicTableSink 
(Base64-encoded)");
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param source}. */
+    public static TableDescriptor forSource(Schema schema, SourceBase source) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SOURCE, source.serialize())
+                .build();
+    }
+
+    /** Creates a {@link TableDescriptor} for the given {@param schema} and 
{@param sink}. */
+    public static TableDescriptor forSink(Schema schema, SinkBase sink) {
+        return TableDescriptor.forConnector(IDENTIFIER)
+                .schema(schema)
+                .option(SINK, sink.serialize())
+                .build();
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /** Harness factory for creating sources / sinks from base64-encoded 
serialized strings. */
+    public static class Factory implements DynamicTableSourceFactory, 
DynamicTableSinkFactory {
+
+        @Override
+        public String factoryIdentifier() {
+            return IDENTIFIER;
+        }
+
+        @Override
+        public Set<ConfigOption<?>> requiredOptions() {
+            return new HashSet<>();
+        }
+
+        @Override
+        public Set<ConfigOption<?>> optionalOptions() {
+            Set<ConfigOption<?>> options = new HashSet<>();
+            options.add(SOURCE);
+            options.add(SINK);
+            return options;
+        }
+
+        @Override
+        public DynamicTableSource createDynamicTableSource(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSource source =
+                    deserializeSourceSink(factoryHelper.getOptions(), SOURCE);
+            if (source instanceof SourceBase) {
+                ((SourceBase) source).factoryContext = context;
+            }
+
+            return source;
+        }
+
+        @Override
+        public DynamicTableSink createDynamicTableSink(Context context) {
+            final FactoryUtil.TableFactoryHelper factoryHelper =
+                    FactoryUtil.createTableFactoryHelper(this, context);
+            factoryHelper.validate();
+
+            final DynamicTableSink sink = 
deserializeSourceSink(factoryHelper.getOptions(), SINK);
+            if (sink instanceof SinkBase) {
+                ((SinkBase) sink).factoryContext = context;
+            }
+
+            return sink;
+        }
+
+        private <T> T deserializeSourceSink(
+                ReadableConfig options, ConfigOption<String> configOption) {
+            final String serializedValue =
+                    options.getOptional(configOption)
+                            .orElseThrow(
+                                    () ->
+                                            new ValidationException(
+                                                    String.format(
+                                                            "Missing option 
'%s'.",
+                                                            
configOption.key())));
+
+            try {
+                return InstantiationUtil.deserializeObject(
+                        Base64.getDecoder().decode(serializedValue),
+                        Thread.currentThread().getContextClassLoader());
+            } catch (ClassNotFoundException | IOException e) {
+                throw new ValidationException(
+                        "Serialized source/sink could not be deserialized.", 
e);
+            }
+        }
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+
+    /**
+     * Serializes a source / sink into a base64-encoded string which can be 
used by {@link Factory}.
+     *
+     * <p>If your source / sink extends from {@link ScanSourceBase}, {@link 
LookupSourceBase}, or
+     * {@link SinkBase}, you can use {@link SourceBase#serialize()} / {@link 
SinkBase#serialize()}
+     * instead, or use {@link #forSource(Schema, SourceBase)} / {@link 
#forSink(Schema, SinkBase)}.
+     */
+    public static String serializeImplementation(Object obj) {
+        try {
+            return 
Base64.getEncoder().encodeToString(InstantiationUtil.serializeObject(obj));
+        } catch (IOException e) {
+            throw new RuntimeException(e.getMessage(), e);
+        }
+    }
+
+    /** Creates a {@link ScanRuntimeProvider} which produces nothing. */
+    public static ScanRuntimeProvider createEmptyScanProvider() {
+        return SourceFunctionProvider.of(
+                new SourceFunction<RowData>() {
+                    @Override
+                    public void run(SourceContext<RowData> ctx) {}
+
+                    @Override
+                    public void cancel() {}
+                },
+                true);

Review comment:
       for `forUnboundedSource ` we can also offer the changelog mode parameter 
already




-- 
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-unsubscr...@flink.apache.org

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


Reply via email to