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



##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TableFactoryHarness.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.ScanTableSource;
+import 
org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+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.testutils.junit.SharedObjects;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Base64;
+import java.util.Collections;
+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 SourceBase} or {@link SinkBase} which provides 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 SourceBase 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() {
+            return Collections.singleton(SOURCE);

Review comment:
       Missing SINK




-- 
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