aalexandrov commented on a change in pull request #13770:
URL: https://github.com/apache/flink/pull/13770#discussion_r517277900



##########
File path: 
flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/table/KinesisDynamicTableFactoryTest.java
##########
@@ -0,0 +1,358 @@
+/*
+ * 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.streaming.connectors.kinesis.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
+import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer;
+import org.apache.flink.streaming.connectors.kinesis.RandomKinesisPartitioner;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TableOptionsBuilder;
+import org.apache.flink.table.factories.TestFormatFactory;
+import 
org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import 
org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
+import static 
org.apache.flink.streaming.connectors.kinesis.table.RowDataKinesisDeserializationSchema.Metadata;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test for {@link KinesisDynamicSource} and {@link KinesisDynamicSink} created
+ * by {@link KinesisDynamicTableFactory}.
+ */
+public class KinesisDynamicTableFactoryTest extends TestLogger {
+
+       private static final String SOURCE_STREAM = "sourceStream";
+       private static final String SINK_STREAM = "targetStream";
+
+       private static final String SOURCE_TABLE = "sourceTable";
+       private static final String SINK_TABLE = "sinkTable";
+
+       private static final Properties KINESIS_PROPERTIES = new Properties() {{
+               setProperty("aws.region", "us-west-2");
+       }};
+
+       @Rule
+       public ExpectedException thrown = ExpectedException.none();
+
+       // 
--------------------------------------------------------------------------------------------
+       // Positive tests
+       // 
--------------------------------------------------------------------------------------------
+
+       @Test
+       public void testGoodTableSource() {
+               TableSchema sourceSchema = defaultSourceSchema().build();
+               Map<String, String> sourceOptions = 
defaultSourceOptions().build();
+
+               // Construct actual DynamicTableSource using FactoryUtil
+               KinesisDynamicSource actualSource = (KinesisDynamicSource)
+                       FactoryUtil.createTableSource(
+                               null,
+                               ObjectIdentifier.of("default", "default", 
SOURCE_TABLE),
+                               createSourceTable(sourceSchema, sourceOptions, 
Collections.emptyList()),
+                               new Configuration(),
+                               Thread.currentThread().getContextClassLoader(),
+                               false);
+
+               // Construct expected DynamicTableSink using factory under test
+               KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+                       sourceSchema.toPhysicalRowDataType(),
+                       SOURCE_STREAM,
+                       KINESIS_PROPERTIES,
+                       new TestFormatFactory.DecodingFormatMock(",", true));
+
+               // verify that the constructed DynamicTableSink is as expected
+               assertEquals(expectedSource, actualSource);
+
+               // verify that the copy of the constructed DynamicTableSink is 
as expected
+               assertEquals(expectedSource, actualSource.copy());
+
+               // verify produced sink
+               ScanTableSource.ScanRuntimeProvider functionProvider =
+                       
actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+               SourceFunction<RowData> sourceFunction =
+                       as(functionProvider, 
SourceFunctionProvider.class).createSourceFunction();
+               assertThat(sourceFunction, 
instanceOf(FlinkKinesisConsumer.class));
+       }
+
+       @Test
+       public void testGoodTableSourceWithMetadataFields() {
+               TableSchema sourceSchema = defaultSourceSchema().build();
+               Map<String, String> sourceOptions = 
defaultSourceOptions().build();
+
+               // Construct actual DynamicTableSource using FactoryUtil
+               KinesisDynamicSource actualSource = (KinesisDynamicSource)
+                       FactoryUtil.createTableSource(
+                               null,
+                               ObjectIdentifier.of("default", "default", 
SOURCE_TABLE),
+                               createSourceTable(sourceSchema, sourceOptions, 
Collections.emptyList()),
+                               new Configuration(),
+                               Thread.currentThread().getContextClassLoader(),
+                               false);
+
+               // Construct expected DynamicTableSink using factory under test
+               KinesisDynamicSource expectedSource = new KinesisDynamicSource(
+                       sourceSchema.toPhysicalRowDataType(),
+                       SOURCE_STREAM,
+                       KINESIS_PROPERTIES,
+                       new TestFormatFactory.DecodingFormatMock(",", true));
+
+               List<String> metadataKeys = Arrays.asList("ShardId", 
"ApproximateArrivalTimestamp");
+               DataType producedDataType = getProducedType(sourceSchema, 
Metadata.values());
+
+               expectedSource.applyReadableMetadata(metadataKeys, 
producedDataType);

Review comment:
       > we should test the contents of mutable member variables 
   
   These are tested by the following `assertEquals(expectedSource, 
actualSource);` call, where `KinesisDynamicSource::equals` is
   
   ```java
   @Override
   public boolean equals(Object o) {
        if (this == o) {
                return true;
        }
        if (o == null || getClass() != o.getClass()) {
                return false;
        }
        KinesisDynamicSource that = (KinesisDynamicSource) o;
        return Objects.equals(producedDataType, that.producedDataType) &&
                Objects.equals(requestedMetadataFields, 
that.requestedMetadataFields) &&
                Objects.equals(physicalDataType, that.physicalDataType) &&
                Objects.equals(stream, that.stream) &&
                Objects.equals(consumerProperties, that.consumerProperties) &&
                Objects.equals(decodingFormat, that.decodingFormat);
   }
   ```
   
   This goes somewhat against [the coding 
guideline](https://flink.apache.org/contributing/code-style-and-quality-java.html#equals--hashcode),
 but it is consistent with other connectors implementing ability interfaces 
(including `KafkaDynamicSource`). For me this is indicative of a broader issue 
with the ability interfaces. IMHO calls of ability interface methods should 
return modified copies of the original source/sink node as opposed to mutating 
that node (this is compatible with the current design of these classes as 
immutable factories).  




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to