wuchong commented on a change in pull request #14727:
URL: https://github.com/apache/flink/pull/14727#discussion_r569977623



##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
##########
@@ -379,6 +390,33 @@ private Object createWriter(Context sinkContext) {
         }
     }
 
+    private void checkAllowanceOfSettingParallelism(
+            int inputParallelism, int configuredParallelism) {
+
+        if (inputParallelism != configuredParallelism) {
+            ChangelogMode mode;
+            if (bulkWriterFormat != null) {
+                mode = bulkWriterFormat.getChangelogMode();
+            } else if (serializationFormat != null) {
+                mode = serializationFormat.getChangelogMode();
+            } else {
+                throw new TableException("Can not find format factory.");
+            }

Review comment:
       Sounds good to me. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
##########
@@ -379,6 +390,33 @@ private Object createWriter(Context sinkContext) {
         }
     }
 
+    private void checkAllowanceOfSettingParallelism(
+            int inputParallelism, int configuredParallelism) {
+
+        if (inputParallelism != configuredParallelism) {
+            ChangelogMode mode;
+            if (bulkWriterFormat != null) {
+                mode = bulkWriterFormat.getChangelogMode();
+            } else if (serializationFormat != null) {
+                mode = serializationFormat.getChangelogMode();
+            } else {
+                throw new TableException("Can not find format factory.");
+            }

Review comment:
       Sounds good to me. You can call it `getFormatChangelogMode()`.

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
##########
@@ -379,6 +390,33 @@ private Object createWriter(Context sinkContext) {
         }
     }
 
+    private void checkAllowanceOfSettingParallelism(
+            int inputParallelism, int configuredParallelism) {
+
+        if (inputParallelism != configuredParallelism) {
+            ChangelogMode mode;
+            if (bulkWriterFormat != null) {
+                mode = bulkWriterFormat.getChangelogMode();
+            } else if (serializationFormat != null) {
+                mode = serializationFormat.getChangelogMode();
+            } else {
+                throw new TableException("Can not find format factory.");
+            }

Review comment:
       I think we shouldn't use the format changelog mode, becuase it can't 
represent input stream changelog mode. A debezium-json can still be used for 
batch mode and we don't need key shuffle in this case. 
   
   I would suggest to check whether paralleslim is configured in 
`DynamicTableSink#getChangelogMode`. We also have a primary key validation 
check in `JdbcDynamicTableSink#getChangelogMode`. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java
##########
@@ -379,6 +390,33 @@ private Object createWriter(Context sinkContext) {
         }
     }
 
+    private void checkAllowanceOfSettingParallelism(
+            int inputParallelism, int configuredParallelism) {
+
+        if (inputParallelism != configuredParallelism) {
+            ChangelogMode mode;
+            if (bulkWriterFormat != null) {
+                mode = bulkWriterFormat.getChangelogMode();
+            } else if (serializationFormat != null) {
+                mode = serializationFormat.getChangelogMode();
+            } else {
+                throw new TableException("Can not find format factory.");
+            }

Review comment:
       1. Yes.
   2. Yes. 
   3. We don't need to check PK. 

##########
File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/FileSystemTableSinkTest.java
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.filesystem;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+/** Test for {@link FileSystemTableSink}. */
+public class FileSystemTableSinkTest {

Review comment:
       1 & 2. you can move the test to `flink-table-planner-blink`.
   3. I'm worried it doesn't verify the parallelism of CompactOperator and 
CompactCoordinator.




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