matriv commented on a change in pull request #17811:
URL: https://github.com/apache/flink/pull/17811#discussion_r751687627



##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/ConstraintValidator.java
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.runtime.operators.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.TableException;
+import 
org.apache.flink.table.api.config.ExecutionConfigOptions.NotNullEnforcer;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static 
org.apache.flink.table.api.config.ExecutionConfigOptions.CharPrecisionEnforcer;
+import static 
org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_CHAR_PRECISION_ENFORCER;
+import static 
org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_NOT_NULL_ENFORCER;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Checks writing null values into NOT NULL columns. */
+@Internal
+public class ConstraintValidator extends TableStreamOperator<RowData>
+        implements OneInputStreamOperator<RowData, RowData> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final NotNullEnforcer notNullEnforcer;
+    private final int[] notNullFieldIndices;
+    private final String[] allFieldNames;
+    private final CharPrecisionEnforcer charPrecisionEnforcer;
+    private final List<Tuple2<Integer, Integer>> charFields;
+
+    private ConstraintValidator(
+            NotNullEnforcer notNullEnforcer,
+            int[] notNullFieldIndices,
+            String[] allFieldNames,
+            CharPrecisionEnforcer charPrecisionEnforcer,
+            List<Tuple2<Integer, Integer>> charFields) {
+        this.notNullFieldIndices = notNullFieldIndices;
+        this.notNullEnforcer = notNullEnforcer;
+        this.allFieldNames = allFieldNames;
+        this.charPrecisionEnforcer = charPrecisionEnforcer;
+        this.charFields = charFields;
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Helper builder, so that the {@link ConstraintValidator} can be 
instantiated with only the NOT
+     * NULL constraint validation, only the CHAR/VARCHAR precision validation, 
or both.
+     */
+    public static class Builder {
+
+        private NotNullEnforcer notNullEnforcer;
+        private int[] notNullFieldIndices;
+        private String[] allFieldNames;
+
+        private CharPrecisionEnforcer charPrecisionEnforcer;
+        private List<Tuple2<Integer, Integer>> charFields;
+
+        private boolean mustApply = false;
+
+        public boolean mustApply() {
+            return mustApply;
+        }
+
+        public void addNotNullConstraint(
+                NotNullEnforcer notNullEnforcer,
+                int[] notNullFieldIndices,
+                String[] allFieldNames) {
+            checkArgument(
+                    notNullFieldIndices.length > 0,
+                    "ConstraintValidator requires that there are not-null 
fields.");
+            this.notNullFieldIndices = notNullFieldIndices;
+            this.notNullEnforcer = notNullEnforcer;
+            this.allFieldNames = allFieldNames;
+            this.mustApply = true;
+        }
+
+        public void addCharPrecisionConstraint(
+                CharPrecisionEnforcer charPrecisionEnforcer,
+                List<Tuple2<Integer, Integer>> charFields,
+                String[] allFieldNames) {
+            checkArgument(
+                    charFields.size() > 0,
+                    "ConstraintValidator requires that there are CHAR/VARCHAR 
fields.");
+            this.charPrecisionEnforcer = charPrecisionEnforcer;
+            this.charFields = charFields;
+            this.allFieldNames = allFieldNames;
+            this.mustApply = true;
+        }
+
+        public ConstraintValidator build() {
+            return new ConstraintValidator(
+                    notNullEnforcer,
+                    notNullFieldIndices,
+                    allFieldNames,
+                    charPrecisionEnforcer,
+                    charFields);
+        }
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        StreamRecord<RowData> processedElement = 
processNotNullConstraint(element);
+        if (processedElement != null) {
+            output.collect(processCharConstraint(processedElement));
+        }
+    }
+
+    private StreamRecord<RowData> 
processNotNullConstraint(StreamRecord<RowData> element) {
+        if (notNullEnforcer == null) {
+            return element;
+        }
+
+        final RowData rowData = element.getValue();
+        for (int index : notNullFieldIndices) {
+            if (rowData.isNullAt(index)) {
+                switch (notNullEnforcer) {
+                    case ERROR:
+                        throw new TableException(
+                                String.format(
+                                        "Column '%s' is NOT NULL, however, a 
null value is being written into it. "
+                                                + "You can set job 
configuration '%s'='drop' "
+                                                + "to suppress this exception 
and drop such records silently.",
+                                        allFieldNames[index],
+                                        
TABLE_EXEC_SINK_NOT_NULL_ENFORCER.key()));
+                    case DROP:
+                        return null;
+                }
+            }
+        }
+        return element;
+    }
+
+    private StreamRecord<RowData> processCharConstraint(StreamRecord<RowData> 
element) {
+        if (charPrecisionEnforcer == null) {
+            return element;
+        }
+
+        final RowData rowData = element.getValue();
+        final List<Tuple2<Integer, String>> newStrings = new ArrayList<>();
+        for (Tuple2<Integer, Integer> tuple : charFields) {
+            final int fieldIdx = tuple.f0;
+            final int length = tuple.f1;
+            final String stringValue = rowData.getString(fieldIdx).toString();
+
+            if (stringValue.length() > length) {
+                switch (charPrecisionEnforcer) {
+                    case ERROR:
+                        throw new TableException(
+                                String.format(
+                                        "Column '%s' is of CHAR/VARCHAR type 
with a length of '%s', however, "
+                                                + "a string value with a 
length of '%s' is being written into it. "
+                                                + "You can set job 
configuration '%s'='trim' to suppress this exception "
+                                                + "and trim such string values 
to the length defined by the CHAR/VARCHAR "
+                                                + "type.",
+                                        allFieldNames[fieldIdx],
+                                        length,
+                                        stringValue.length(),
+                                        
TABLE_EXEC_SINK_CHAR_PRECISION_ENFORCER.key()));
+                    case TRIM:
+                        newStrings.add(new Tuple2<>(fieldIdx, 
stringValue.substring(0, length)));

Review comment:
       We can do it here and not use a `List<Tuple2<>>`, but how can we reuse 
the `RowData`? that's the `RowData` iface so we don't know the implementation 
and call directly `.setField()` on it. Could you please provide more details, 
probably I'm missing something.




-- 
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: [email protected]

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


Reply via email to