luoyuxia commented on code in PR #21977: URL: https://github.com/apache/flink/pull/21977#discussion_r1112798182
########## flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableCompactSinkTest.java: ########## @@ -18,178 +18,174 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.file.table.FileSystemConnectorOptions; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.file.table.batch.BatchSink; -import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.api.graph.StreamNode; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.CatalogTest; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.table.catalog.hive.HiveTestUtils; -import org.apache.flink.table.connector.sink.DataStreamSinkProvider; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.util.TestLoggerExtension; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; +import java.util.List; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.COMPACTION_PARALLELISM; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARALLELISM; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link HiveTableSink} enable auto-compaction. */ @ExtendWith(TestLoggerExtension.class) class HiveTableCompactSinkTest { Review Comment: Since it's mainly for test parallelism, how about rename it to `HiveTableCompactSinkParallelismTest`? Also the java doc for this class should update. ########## flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableCompactSinkTest.java: ########## @@ -18,178 +18,174 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.file.table.FileSystemConnectorOptions; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.file.table.batch.BatchSink; -import org.apache.flink.sql.parser.hive.ddl.SqlCreateHiveTable; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.api.graph.StreamNode; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.CatalogTest; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.catalog.hive.HiveCatalog; import org.apache.flink.table.catalog.hive.HiveTestUtils; -import org.apache.flink.table.connector.sink.DataStreamSinkProvider; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.util.TestLoggerExtension; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; +import java.util.List; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.COMPACTION_PARALLELISM; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARALLELISM; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link HiveTableSink} enable auto-compaction. */ @ExtendWith(TestLoggerExtension.class) class HiveTableCompactSinkTest { + /** + * Represents the parallelism need not check, it should follow the setting of planer or auto Review Comment: nit: ```suggestion * Represents the parallelism doesn't need to be checked, it should follow the setting of planer or auto ``` ########## flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableCompactSinkTest.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.connector.file.table.batch.BatchSink; +import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.catalog.hive.HiveCatalog; +import org.apache.flink.table.catalog.hive.HiveTestUtils; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.COMPACTION_PARALLELISM; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARALLELISM; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link HiveTableSink} enable auto-compaction. */ +@ExtendWith(TestLoggerExtension.class) +class HiveTableCompactSinkTest { + /** + * Represents the parallelism need not check, it should follow the setting of planer or auto + * inference. + */ + public static final int NEED_NOT_CHECK_PARALLELISM = -1; + + private HiveCatalog catalog; + + private TableEnvironment tableEnv; + + @BeforeEach + void before() { + catalog = HiveTestUtils.createHiveCatalog(); + catalog.open(); + tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); + tableEnv.registerCatalog(catalog.getName(), catalog); + tableEnv.useCatalog(catalog.getName()); + } + + @AfterEach + void after() { + if (catalog != null) { + catalog.close(); + } + } + + /** If only sink parallelism is set, compact operator should follow this setting. */ + @Test + void testOnlySetSinkParallelism() { + final int sinkParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s' )", + SINK_PARALLELISM.key(), sinkParallelism)); + + assertSinkAndCompactOperatorParallelism(true, true, sinkParallelism, sinkParallelism); + } + + @Test + void testOnlySetCompactParallelism() { + final int compactParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s' )", + COMPACTION_PARALLELISM.key(), compactParallelism)); + + assertSinkAndCompactOperatorParallelism( + false, true, NEED_NOT_CHECK_PARALLELISM, compactParallelism); + } + + @Test + void testSetBothSinkAndCompactParallelism() { + final int sinkParallelism = 8; + final int compactParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s', " + + " '%s' = '%s' )", + SINK_PARALLELISM.key(), + sinkParallelism, + COMPACTION_PARALLELISM.key(), + compactParallelism)); + + assertSinkAndCompactOperatorParallelism(true, true, sinkParallelism, compactParallelism); + } + + @Test + void testSinkAndCompactAllNotSetParallelism() { + tableEnv.executeSql( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true' )"); + assertSinkAndCompactOperatorParallelism( + false, false, NEED_NOT_CHECK_PARALLELISM, NEED_NOT_CHECK_PARALLELISM); + } + + private void assertSinkAndCompactOperatorParallelism( + boolean isSinkParallelismConfigured, + boolean isCompactParallelismConfigured, + int expectedSinkParallelism, + int expectedCompactParallelism) { + String statement = "insert into src values ('k1', 'v1'), ('k2', 'v2');"; + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) tableEnv).getPlanner(); + planner.getExecEnv().setParallelism(10); + List<Operation> operations = planner.getParser().parse(statement); + List<Transformation<?>> transformations = + planner.translate(Collections.singletonList((ModifyOperation) (operations.get(0)))); + assertThat(transformations).hasSize(1); + Transformation<?> rootTransformation = transformations.get(0); + Transformation<?> compactTransformation = + findTransformationByName(rootTransformation, BatchSink.COMPACT_OP_NAME); + Transformation<?> hiveSinkTransformation = + findTransformationByName( + rootTransformation, HiveTableSink.BATCH_COMPACT_WRITER_OP_NAME); + assertThat(hiveSinkTransformation.isParallelismConfigured()) + .isEqualTo(isSinkParallelismConfigured); + assertThat(compactTransformation.isParallelismConfigured()) + .isEqualTo(isCompactParallelismConfigured); + if (isSinkParallelismConfigured) { Review Comment: Got little confused in here. I guess `NEED_NOT_CHECK_PARALLELISM` means if the passed `expectedSinkParallelism` is equal to `NEED_NOT_CHECK_PARALLELISM` , we can ignore check whether they are same. If that's the case, won't it be more clear with the following code? ``` if (expectedSinkParallelism != NEED_NOT_CHECK_PARALLELISM) { // then we check the parallelism }. ``` ########## flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableCompactSinkTest.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.connectors.hive; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.connector.file.table.batch.BatchSink; +import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.catalog.hive.HiveCatalog; +import org.apache.flink.table.catalog.hive.HiveTestUtils; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.COMPACTION_PARALLELISM; +import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARALLELISM; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link HiveTableSink} enable auto-compaction. */ +@ExtendWith(TestLoggerExtension.class) +class HiveTableCompactSinkTest { + /** + * Represents the parallelism need not check, it should follow the setting of planer or auto + * inference. + */ + public static final int NEED_NOT_CHECK_PARALLELISM = -1; + + private HiveCatalog catalog; + + private TableEnvironment tableEnv; + + @BeforeEach + void before() { + catalog = HiveTestUtils.createHiveCatalog(); + catalog.open(); + tableEnv = HiveTestUtils.createTableEnvInBatchMode(SqlDialect.HIVE); + tableEnv.registerCatalog(catalog.getName(), catalog); + tableEnv.useCatalog(catalog.getName()); + } + + @AfterEach + void after() { + if (catalog != null) { + catalog.close(); + } + } + + /** If only sink parallelism is set, compact operator should follow this setting. */ + @Test + void testOnlySetSinkParallelism() { + final int sinkParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s' )", + SINK_PARALLELISM.key(), sinkParallelism)); + + assertSinkAndCompactOperatorParallelism(true, true, sinkParallelism, sinkParallelism); + } + + @Test + void testOnlySetCompactParallelism() { + final int compactParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s' )", + COMPACTION_PARALLELISM.key(), compactParallelism)); + + assertSinkAndCompactOperatorParallelism( + false, true, NEED_NOT_CHECK_PARALLELISM, compactParallelism); + } + + @Test + void testSetBothSinkAndCompactParallelism() { + final int sinkParallelism = 8; + final int compactParallelism = 4; + + tableEnv.executeSql( + String.format( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true', " + + " '%s' = '%s', " + + " '%s' = '%s' )", + SINK_PARALLELISM.key(), + sinkParallelism, + COMPACTION_PARALLELISM.key(), + compactParallelism)); + + assertSinkAndCompactOperatorParallelism(true, true, sinkParallelism, compactParallelism); + } + + @Test + void testSinkAndCompactAllNotSetParallelism() { + tableEnv.executeSql( + "CREATE TABLE src (" + + " key string," + + " value string" + + ") TBLPROPERTIES (" + + " 'auto-compaction' = 'true' )"); + assertSinkAndCompactOperatorParallelism( + false, false, NEED_NOT_CHECK_PARALLELISM, NEED_NOT_CHECK_PARALLELISM); + } + + private void assertSinkAndCompactOperatorParallelism( + boolean isSinkParallelismConfigured, + boolean isCompactParallelismConfigured, + int expectedSinkParallelism, + int expectedCompactParallelism) { + String statement = "insert into src values ('k1', 'v1'), ('k2', 'v2');"; + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) tableEnv).getPlanner(); + planner.getExecEnv().setParallelism(10); + List<Operation> operations = planner.getParser().parse(statement); + List<Transformation<?>> transformations = + planner.translate(Collections.singletonList((ModifyOperation) (operations.get(0)))); + assertThat(transformations).hasSize(1); + Transformation<?> rootTransformation = transformations.get(0); + Transformation<?> compactTransformation = + findTransformationByName(rootTransformation, BatchSink.COMPACT_OP_NAME); + Transformation<?> hiveSinkTransformation = + findTransformationByName( + rootTransformation, HiveTableSink.BATCH_COMPACT_WRITER_OP_NAME); + assertThat(hiveSinkTransformation.isParallelismConfigured()) + .isEqualTo(isSinkParallelismConfigured); + assertThat(compactTransformation.isParallelismConfigured()) + .isEqualTo(isCompactParallelismConfigured); + if (isSinkParallelismConfigured) { Review Comment: If that's not what `NEED_NOT_CHECK_PARALLELISM` means , then I think it's just a place holder? If so, seems it'll be better to update the name/comment of `NEED_NOT_CHECK_PARALLELISM` to make it more clear. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org