[ 
https://issues.apache.org/jira/browse/FLINK-10061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16594706#comment-16594706
 ] 

ASF GitHub Bot commented on FLINK-10061:
----------------------------------------

tragicjun closed pull request #6495: [FLINK-10061] [table] [kafka] Fix 
unsupported reconfiguration in KafkaTableSink
URL: https://github.com/apache/flink/pull/6495
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
 
b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
index a85d536eac9..772a94a2c4f 100644
--- 
a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
+++ 
b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
@@ -26,6 +26,7 @@
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import 
org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.sinks.AppendStreamTableSink;
 import org.apache.flink.table.util.TableConnectorUtil;
 import org.apache.flink.types.Row;
@@ -166,8 +167,15 @@ public void emitDataStream(DataStream<Row> dataStream) {
        @Override
        public KafkaTableSink configure(String[] fieldNames, 
TypeInformation<?>[] fieldTypes) {
                if (schema.isPresent()) {
-                       // a fixed schema is defined so reconfiguration is not 
supported
-                       throw new 
UnsupportedOperationException("Reconfiguration of this sink is not supported.");
+                       if (getFieldTypes().length != fieldTypes.length || 
!Arrays.deepEquals(getFieldTypes(), fieldTypes)) {
+                               String fixedFieldTypes = 
Arrays.toString(getFieldTypes());
+                               String configuredFieldTypes = 
Arrays.toString(fieldTypes);
+                               throw new ValidationException("A fixed table 
schema is defined but the configured fieldTypes " +
+                                               configuredFieldTypes + " does 
not match with the fieldTypes " + fixedFieldTypes +
+                                               " declared by the fixed table 
schema.");
+                       }
+
+                       return this;
                }
 
                // legacy code


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Fix unsupported reconfiguration in KafkaTableSink
> -------------------------------------------------
>
>                 Key: FLINK-10061
>                 URL: https://issues.apache.org/jira/browse/FLINK-10061
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.6.0
>            Reporter: Jun Zhang
>            Assignee: Jun Zhang
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.6.1, 1.7.0
>
>
> When using KafkaTableSink in "table.writeToSink(), the following exception is 
> thrown:
> {quote} java.lang.UnsupportedOperationException: Reconfiguration of this sink 
> is not supported.
> {quote}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to