|
| 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, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.hudi.sink; |
| 20 | + |
| 21 | +import org.apache.hudi.client.common.HoodieFlinkEngineContext; |
| 22 | +import org.apache.hudi.common.model.FileSlice; |
| 23 | +import org.apache.hudi.common.model.HoodieKey; |
| 24 | +import org.apache.hudi.common.model.HoodieRecord; |
| 25 | +import org.apache.hudi.common.model.HoodieTableType; |
| 26 | +import org.apache.hudi.common.table.TableSchemaResolver; |
| 27 | +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; |
| 28 | +import org.apache.hudi.common.table.timeline.HoodieInstant; |
| 29 | +import org.apache.hudi.common.table.timeline.HoodieTimeline; |
| 30 | +import org.apache.hudi.common.util.BaseFileUtils; |
| 31 | +import org.apache.hudi.common.util.ClosableIterator; |
| 32 | +import org.apache.hudi.common.util.Option; |
| 33 | +import org.apache.hudi.config.HoodieWriteConfig; |
| 34 | +import org.apache.hudi.configuration.FlinkOptions; |
| 35 | +import org.apache.hudi.exception.HoodieException; |
| 36 | +import org.apache.hudi.sink.transform.Transformer; |
| 37 | +import org.apache.hudi.sink.utils.Pipelines; |
| 38 | +import org.apache.hudi.table.HoodieFlinkTable; |
| 39 | +import org.apache.hudi.table.format.FormatUtils; |
| 40 | +import org.apache.hudi.util.AvroSchemaConverter; |
| 41 | +import org.apache.hudi.util.FlinkClientUtil; |
| 42 | +import org.apache.hudi.util.StreamerUtil; |
| 43 | +import org.apache.hudi.utils.TestConfigurations; |
| 44 | +import org.apache.hudi.utils.source.ContinuousFileSource; |
| 45 | + |
| 46 | +import org.apache.avro.Schema; |
| 47 | +import org.apache.flink.configuration.Configuration; |
| 48 | +import org.apache.flink.core.execution.JobClient; |
| 49 | +import org.apache.flink.core.fs.Path; |
| 50 | +import org.apache.flink.formats.common.TimestampFormat; |
| 51 | +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; |
| 52 | +import org.apache.flink.streaming.api.CheckpointingMode; |
| 53 | +import org.apache.flink.streaming.api.datastream.DataStream; |
| 54 | +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 55 | +import org.apache.flink.table.data.RowData; |
| 56 | +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; |
| 57 | +import org.apache.flink.table.types.logical.RowType; |
| 58 | +import org.apache.flink.util.TestLogger; |
| 59 | +import org.junit.jupiter.api.Test; |
| 60 | +import org.junit.jupiter.api.io.TempDir; |
| 61 | + |
| 62 | +import java.io.File; |
| 63 | +import java.io.FilenameFilter; |
| 64 | +import java.nio.charset.StandardCharsets; |
| 65 | +import java.util.List; |
| 66 | +import java.util.Objects; |
| 67 | +import java.util.concurrent.atomic.AtomicInteger; |
| 68 | + |
| 69 | +import static java.util.stream.Collectors.toList; |
| 70 | +import static org.apache.hudi.util.StreamerUtil.isValidFile; |
| 71 | +import static org.junit.jupiter.api.Assertions.assertEquals; |
| 72 | + |
| 73 | +/** |
| 74 | + * Integration test for BoostrapOperator. |
| 75 | + */ |
| 76 | +public class TestBoostrapOperator extends TestLogger { |
| 77 | + @TempDir |
| 78 | + File tempFile; |
| 79 | + |
| 80 | + @Test |
| 81 | + public void testLoadRecords() throws Exception { |
| 82 | + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); |
| 83 | + conf.setInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS, 4); |
| 84 | + conf.setString(FlinkOptions.INDEX_KEY_FIELD, "id"); |
| 85 | + conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1); |
| 86 | + conf.setString(FlinkOptions.TABLE_TYPE, HoodieTableType.MERGE_ON_READ.name()); |
| 87 | + |
| 88 | + testWriteToHoodie(conf, Option.empty(), "mor_write_with_compact", 5); |
| 89 | + |
| 90 | + deleteLastCompactionCommit(); |
| 91 | + |
| 92 | + HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(conf, true); |
| 93 | + HoodieFlinkTable hoodieTable = HoodieFlinkTable.create(writeConfig, HoodieFlinkEngineContext.DEFAULT); |
| 94 | + HoodieTimeline commitsTimeline = hoodieTable.getMetaClient().getCommitsTimeline(); |
| 95 | + Option<HoodieInstant> latestCommitTime = commitsTimeline.filterCompletedInstants().lastInstant(); |
| 96 | + AtomicInteger count = new AtomicInteger(); |
| 97 | + BaseFileUtils fileUtils = BaseFileUtils.getInstance(hoodieTable.getBaseFileFormat()); |
| 98 | + Schema schema = new TableSchemaResolver(hoodieTable.getMetaClient()).getTableAvroSchema(); |
| 99 | + if (latestCommitTime.isPresent()) { |
| 100 | + List<FileSlice> fileSlices = hoodieTable.getSliceView() |
| 101 | + .getLatestFileSlicesBeforeOrOn("par1", latestCommitTime.get().getTimestamp(), true, true) |
| 102 | + .collect(toList()); |
| 103 | + for (FileSlice fileSlice : fileSlices) { |
| 104 | + fileSlice.getBaseFile().ifPresent(baseFile -> { |
| 105 | + // filter out crushed files |
| 106 | + if (!isValidFile(baseFile.getFileStatus())) { |
| 107 | + return; |
| 108 | + } |
| 109 | + try (ClosableIterator<HoodieKey> iterator = fileUtils.getHoodieKeyIterator(FlinkClientUtil.getHadoopConf(), new org.apache.hadoop.fs.Path(baseFile.getPath()))) { |
| 110 | + iterator.forEachRemaining(hoodieKey -> { |
| 111 | + count.getAndIncrement(); |
| 112 | + }); |
| 113 | + } |
| 114 | + }); |
| 115 | + |
| 116 | + // load avro log records |
| 117 | + List<String> logPaths = fileSlice.getLogFiles() |
| 118 | + // filter out crushed files |
| 119 | + .filter(logFile -> isValidFile(logFile.getFileStatus())) |
| 120 | + .map(logFile -> logFile.getPath().toString()) |
| 121 | + .collect(toList()); |
| 122 | + HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(logPaths, schema, latestCommitTime.get().getTimestamp(), |
| 123 | + writeConfig, FlinkClientUtil.getHadoopConf()); |
| 124 | + |
| 125 | + try { |
| 126 | + for (String recordKey : scanner.getRecords().keySet()) { |
| 127 | + count.getAndIncrement(); |
| 128 | + } |
| 129 | + } catch (Exception e) { |
| 130 | + throw new HoodieException(String.format("Error when loading record keys from files: %s", logPaths), e); |
| 131 | + } finally { |
| 132 | + scanner.close(); |
| 133 | + } |
| 134 | + } |
| 135 | + } |
| 136 | + assertEquals(8, count.get()); |
| 137 | + } |
| 138 | + |
| 139 | + private void deleteLastCompactionCommit() { |
| 140 | + File allCommits = new File(tempFile.getPath(), ".hoodie"); |
| 141 | + final File[] files = allCommits.listFiles(new FilenameFilter() { |
| 142 | + @Override |
| 143 | + public boolean accept(File dir, String name) { |
| 144 | + return name.endsWith(".commit"); |
| 145 | + } |
| 146 | + }); |
| 147 | + if (files.length > 0) { |
| 148 | + files[files.length - 1].delete(); |
| 149 | + } |
| 150 | + } |
| 151 | + |
| 152 | + private void testWriteToHoodie( |
| 153 | + Configuration conf, |
| 154 | + Option<Transformer> transformer, |
| 155 | + String jobName, |
| 156 | + int checkpoints) throws Exception { |
| 157 | + |
| 158 | + StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); |
| 159 | + execEnv.getConfig().disableObjectReuse(); |
| 160 | + execEnv.setParallelism(4); |
| 161 | + // set up checkpoint interval |
| 162 | + execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE); |
| 163 | + execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1); |
| 164 | + |
| 165 | + // Read from file source |
| 166 | + RowType rowType = |
| 167 | + (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) |
| 168 | + .getLogicalType(); |
| 169 | + |
| 170 | + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( |
| 171 | + rowType, |
| 172 | + InternalTypeInfo.of(rowType), |
| 173 | + false, |
| 174 | + true, |
| 175 | + TimestampFormat.ISO_8601 |
| 176 | + ); |
| 177 | + String sourcePath = Objects.requireNonNull(Thread.currentThread() |
| 178 | + .getContextClassLoader().getResource("test_source6.data")).toString(); |
| 179 | + |
| 180 | + DataStream<RowData> dataStream; |
| 181 | + |
| 182 | + dataStream = execEnv |
| 183 | + // use continuous file source to trigger checkpoint |
| 184 | + .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), checkpoints)) |
| 185 | + .name("continuous_file_source") |
| 186 | + .setParallelism(1) |
| 187 | + .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) |
| 188 | + .setParallelism(1); |
| 189 | + |
| 190 | + if (transformer.isPresent()) { |
| 191 | + dataStream = transformer.get().apply(dataStream); |
| 192 | + } |
| 193 | + |
| 194 | + int parallelism = execEnv.getParallelism(); |
| 195 | + DataStream<HoodieRecord> hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, parallelism, dataStream); |
| 196 | + DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream); |
| 197 | + execEnv.addOperator(pipeline.getTransformation()); |
| 198 | + |
| 199 | + Pipelines.clean(conf, pipeline); |
| 200 | + Pipelines.compact(conf, pipeline); |
| 201 | + |
| 202 | + JobClient client = execEnv.executeAsync(jobName); |
| 203 | + // wait for the streaming job to finish |
| 204 | + client.getJobExecutionResult().get(); |
| 205 | + } |
| 206 | +} |
0 commit comments