twalthr commented on code in PR #26540: URL: https://github.com/apache/flink/pull/26540#discussion_r2081370780
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/constraint/Constraint.java: ########## @@ -0,0 +1,34 @@ +/* + * 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.constraint; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +/** Interface for constraints to be enforced on the input {@link RowData}. */ +@Internal +public interface Constraint extends Serializable { + + @Nullable + RowData enforce(RowData input); Review Comment: document what returning null means. esp. for nested structures. ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/constraint/NotNullConstraint.java: ########## @@ -0,0 +1,73 @@ +/* + * 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.constraint; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_NOT_NULL_ENFORCER; + +/** Enforces NOT NULL constraints on the input {@link RowData}. */ +@Internal +final class NotNullConstraint implements Constraint { + private final ExecutionConfigOptions.NotNullEnforcer notNullEnforcer; + private final int[] notNullFieldIndices; + private final String[] notNullFieldNames; + + NotNullConstraint( + ExecutionConfigOptions.NotNullEnforcer notNullEnforcer, + int[] notNullFieldIndices, + String[] notNullFieldNames) { + this.notNullEnforcer = notNullEnforcer; + this.notNullFieldIndices = notNullFieldIndices; + this.notNullFieldNames = notNullFieldNames; + } + + @Nullable + @Override + public RowData enforce(RowData input) { + for (int i = 0; i < notNullFieldIndices.length; i++) { + final int index = notNullFieldIndices[i]; + if (input.isNullAt(index)) { Review Comment: should we also test nullability on elements of maps and arrays? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/constraint/CharLengthConstraint.java: ########## @@ -0,0 +1,139 @@ +/* + * 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.constraint; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +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.data.binary.BinaryStringData; +import org.apache.flink.table.runtime.util.SegmentsUtil; + +import javax.annotation.Nullable; + +import java.util.BitSet; + +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER; + +/** Enforces length constraints on the input {@link RowData}. */ +@Internal +final class CharLengthConstraint implements Constraint { + + private final ExecutionConfigOptions.TypeLengthEnforcer typeLengthEnforcer; Review Comment: nit: maybe introduce your own enum here to decouple runtime from config options for serializability ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java: ########## @@ -235,31 +236,23 @@ void testCharLengthEnforcer() throws ExecutionException, InterruptedException { // Change config option to "trim_pad", to trim or pad the strings // accordingly, based on their type length - try { - tableEnv.getConfig() - .set( - TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER.key(), - ExecutionConfigOptions.TypeLengthEnforcer.TRIM_PAD.name()); - - result = tableEnv.executeSql("SELECT * FROM T1"); - result.await(); - - final List<Row> expected = - Arrays.asList( - Row.of(1, "Apache F", "SQL Ru", 11, 111, "SQL"), - Row.of(2, "Apache ", "SQL ", 22, 222, "Flink"), - Row.of(3, "Apache ", "Flink ", 33, 333, "Apache"), - Row.of(4, "Flink Pr", "SQL or", 44, 444, "Apache")); - final List<Row> resultsTrimmed = new ArrayList<>(); - result.collect().forEachRemaining(resultsTrimmed::add); - assertThat(resultsTrimmed).containsExactlyInAnyOrderElementsOf(expected); - - } finally { - tableEnv.getConfig() - .set( - TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER.key(), - ExecutionConfigOptions.TypeLengthEnforcer.IGNORE.name()); - } + tableEnv.getConfig() + .set( + TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER.key(), + ExecutionConfigOptions.TypeLengthEnforcer.TRIM_PAD.name()); + + result = tableEnv.executeSql("SELECT * FROM T1"); + result.await(); + + final List<Row> expected = + Arrays.asList( + Row.of(1, "Apache F", "SQL Ru", 11, 111, "SQL"), + Row.of(2, "Apache ", "SQL ", 22, 222, "Flink"), + Row.of(3, "Apache ", "Flink ", 33, 333, "Apache"), + Row.of(4, "Flink Pr", "SQL or", 44, 444, "Apache")); + final List<Row> resultsTrimmed = new ArrayList<>(); + result.collect().forEachRemaining(resultsTrimmed::add); Review Comment: Use SemanticTests infra instead? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/constraint/NestedConstraint.java: ########## @@ -0,0 +1,99 @@ +/* + * 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.constraint; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.UpdatableRowData; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** Checks constraints on nested rows. */ +@Internal +final class NestedConstraint implements Constraint { + + private final int[] nestedRowFieldIndices; + private final int[] nestedRowFieldArities; + private final String[] nestedRowFieldNames; + private final List<List<Constraint>> nestedRowConstraints; Review Comment: ```suggestion private final List<List<Constraint>> nestedRowConstraints; ``` nit: maybe use arrays here to make sure this is always serializable -- 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]
