|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.apache.iceberg.flink.source; |
| 20 | + |
| 21 | +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; |
| 22 | +import static org.assertj.core.api.Assertions.assertThat; |
| 23 | + |
| 24 | +import java.io.IOException; |
| 25 | +import java.lang.reflect.Field; |
| 26 | +import java.nio.file.Path; |
| 27 | +import java.util.List; |
| 28 | +import org.apache.flink.configuration.Configuration; |
| 29 | +import org.apache.flink.core.execution.JobClient; |
| 30 | +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; |
| 31 | +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; |
| 32 | +import org.apache.flink.runtime.minicluster.MiniCluster; |
| 33 | +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; |
| 34 | +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; |
| 35 | +import org.apache.flink.streaming.api.datastream.DataStream; |
| 36 | +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 37 | +import org.apache.flink.test.junit5.MiniClusterExtension; |
| 38 | +import org.apache.flink.types.Row; |
| 39 | +import org.apache.flink.util.CloseableIterator; |
| 40 | +import org.apache.iceberg.FileFormat; |
| 41 | +import org.apache.iceberg.Table; |
| 42 | +import org.apache.iceberg.data.GenericAppenderHelper; |
| 43 | +import org.apache.iceberg.data.RandomGenericData; |
| 44 | +import org.apache.iceberg.data.Record; |
| 45 | +import org.apache.iceberg.flink.FlinkConfigOptions; |
| 46 | +import org.apache.iceberg.flink.FlinkSchemaUtil; |
| 47 | +import org.apache.iceberg.flink.HadoopCatalogExtension; |
| 48 | +import org.apache.iceberg.flink.TestFixtures; |
| 49 | +import org.apache.iceberg.flink.data.RowDataToRowMapper; |
| 50 | +import org.apache.iceberg.relocated.com.google.common.collect.Lists; |
| 51 | +import org.junit.jupiter.api.AfterEach; |
| 52 | +import org.junit.jupiter.api.BeforeEach; |
| 53 | +import org.junit.jupiter.api.Test; |
| 54 | +import org.junit.jupiter.api.extension.RegisterExtension; |
| 55 | +import org.junit.jupiter.api.io.TempDir; |
| 56 | + |
| 57 | +public class TestIcebergSourceInferParallelism { |
| 58 | + private static final int NUM_TMS = 2; |
| 59 | + private static final int SLOTS_PER_TM = 2; |
| 60 | + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; |
| 61 | + private static final int MAX_INFERRED_PARALLELISM = 3; |
| 62 | + |
| 63 | + @RegisterExtension |
| 64 | + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = |
| 65 | + new MiniClusterExtension( |
| 66 | + new MiniClusterResourceConfiguration.Builder() |
| 67 | + .setNumberTaskManagers(NUM_TMS) |
| 68 | + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) |
| 69 | + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) |
| 70 | + .build()); |
| 71 | + |
| 72 | + @RegisterExtension |
| 73 | + protected static final HadoopCatalogExtension CATALOG_EXTENSION = |
| 74 | + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); |
| 75 | + |
| 76 | + @TempDir private Path tmpDir; |
| 77 | + |
| 78 | + private Table table; |
| 79 | + private GenericAppenderHelper dataAppender; |
| 80 | + |
| 81 | + @BeforeEach |
| 82 | + public void before() throws IOException { |
| 83 | + this.table = |
| 84 | + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); |
| 85 | + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); |
| 86 | + } |
| 87 | + |
| 88 | + @AfterEach |
| 89 | + public void after() { |
| 90 | + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); |
| 91 | + } |
| 92 | + |
| 93 | + @Test |
| 94 | + public void testEmptyTable() throws Exception { |
| 95 | + // Inferred parallelism should be at least 1 even if table is empty |
| 96 | + test(1, 0); |
| 97 | + } |
| 98 | + |
| 99 | + @Test |
| 100 | + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { |
| 101 | + // Append files to the table |
| 102 | + for (int i = 0; i < 2; ++i) { |
| 103 | + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); |
| 104 | + dataAppender.appendToTable(batch); |
| 105 | + } |
| 106 | + |
| 107 | + // Inferred parallelism should equal to 2 splits |
| 108 | + test(2, 2); |
| 109 | + } |
| 110 | + |
| 111 | + @Test |
| 112 | + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { |
| 113 | + // Append files to the table |
| 114 | + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { |
| 115 | + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); |
| 116 | + dataAppender.appendToTable(batch); |
| 117 | + } |
| 118 | + |
| 119 | + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM |
| 120 | + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); |
| 121 | + } |
| 122 | + |
| 123 | + private void test(int expectedParallelism, int expectedRecords) throws Exception { |
| 124 | + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); |
| 125 | + env.setParallelism(PARALLELISM); |
| 126 | + |
| 127 | + Configuration config = new Configuration(); |
| 128 | + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); |
| 129 | + config.set( |
| 130 | + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, |
| 131 | + MAX_INFERRED_PARALLELISM); |
| 132 | + |
| 133 | + DataStream<Row> dataStream = |
| 134 | + IcebergSource.forRowData() |
| 135 | + .tableLoader(CATALOG_EXTENSION.tableLoader()) |
| 136 | + .table(table) |
| 137 | + .flinkConfig(config) |
| 138 | + // force one file per split |
| 139 | + .splitSize(1L) |
| 140 | + .buildStream(env) |
| 141 | + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); |
| 142 | + |
| 143 | + DataStream.Collector<Row> collector = new DataStream.Collector<>(); |
| 144 | + dataStream.collectAsync(collector); |
| 145 | + JobClient jobClient = env.executeAsync(); |
| 146 | + try (CloseableIterator<Row> iterator = collector.getOutput()) { |
| 147 | + List<Row> result = Lists.newArrayList(); |
| 148 | + while (iterator.hasNext()) { |
| 149 | + result.add(iterator.next()); |
| 150 | + } |
| 151 | + |
| 152 | + assertThat(result).hasSize(expectedRecords); |
| 153 | + verifySourceParallelism( |
| 154 | + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); |
| 155 | + } |
| 156 | + } |
| 157 | + |
| 158 | + /** |
| 159 | + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism |
| 160 | + * from execution graph. |
| 161 | + */ |
| 162 | + private static void verifySourceParallelism( |
| 163 | + int expectedParallelism, AccessExecutionGraph executionGraph) { |
| 164 | + AccessExecutionJobVertex sourceVertex = |
| 165 | + executionGraph.getVerticesTopologically().iterator().next(); |
| 166 | + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); |
| 167 | + } |
| 168 | + |
| 169 | + /** |
| 170 | + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get |
| 171 | + * execution graph and source parallelism. Haven't find other way via public APIS. |
| 172 | + */ |
| 173 | + private static MiniCluster miniCluster() throws Exception { |
| 174 | + Field privateField = |
| 175 | + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); |
| 176 | + privateField.setAccessible(true); |
| 177 | + InternalMiniClusterExtension internalExtension = |
| 178 | + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); |
| 179 | + return internalExtension.getMiniCluster(); |
| 180 | + } |
| 181 | +} |
0 commit comments