diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java index 5d808e48ff94f..3f0fcf9415678 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.util.Option; import java.io.Serializable; +import java.util.List; import java.util.Objects; import java.util.TreeSet; import java.util.stream.Stream; @@ -71,6 +72,15 @@ public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) { this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); } + public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime, + HoodieBaseFile baseFile, List logFiles) { + this.fileGroupId = fileGroupId; + this.baseInstantTime = baseInstantTime; + this.baseFile = baseFile; + this.logFiles = new TreeSet<>(HoodieLogFile.getReverseLogFileComparator()); + this.logFiles.addAll(logFiles); + } + public void setBaseFile(HoodieBaseFile baseFile) { this.baseFile = baseFile; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index 41d83813f1e0f..8568e9926a4bc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -24,9 +24,13 @@ import org.apache.hudi.common.util.collection.Pair; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -37,6 +41,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -236,6 +241,44 @@ public static T fromJsonString(String jsonStr, Class clazz) throws Except return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); } + /** + * parse the bytes of deltacommit, and get the base file and the log files belonging to this + * provided file group. + */ + // TODO: refactor this method to avoid doing the json tree walking (HUDI-4822). + public static Option>> getFileSliceForFileGroupFromDeltaCommit( + byte[] bytes, HoodieFileGroupId fileGroupId) { + String jsonStr = new String(bytes, StandardCharsets.UTF_8); + if (jsonStr.isEmpty()) { + return Option.empty(); + } + + try { + JsonNode ptToWriteStatsMap = JsonUtils.getObjectMapper().readTree(jsonStr).get("partitionToWriteStats"); + Iterator> pts = ptToWriteStatsMap.fields(); + while (pts.hasNext()) { + Map.Entry ptToWriteStats = pts.next(); + if (ptToWriteStats.getValue().isArray()) { + for (JsonNode writeStat : ptToWriteStats.getValue()) { + HoodieFileGroupId fgId = new HoodieFileGroupId(ptToWriteStats.getKey(), writeStat.get("fileId").asText()); + if (fgId.equals(fileGroupId)) { + String baseFile = writeStat.get("baseFile").asText(); + ArrayNode logFilesNode = (ArrayNode) writeStat.get("logFiles"); + List logFiles = new ArrayList<>(); + for (JsonNode logFile : logFilesNode) { + logFiles.add(logFile.asText()); + } + return Option.of(Pair.of(baseFile, logFiles)); + } + } + } + } + return Option.empty(); + } catch (Exception e) { + throw new HoodieException("Fail to parse the base file and log files from DeltaCommit", e); + } + } + // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. public long fetchTotalPartitionsWritten() { return partitionToWriteStats.size(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java new file mode 100644 index 0000000000000..8cae90d59a35e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java @@ -0,0 +1,355 @@ +/* + * 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.hudi.common.table.cdc; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Locale; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_INSERT; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.AS_IS; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.LOG_FILE; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_DELETE; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.REPLACE_COMMIT; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.isInRange; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; + +/** + * This class helps to extract all the information which will be used when CDC query. + * + * There are some steps: + * 1. filter out the completed commit instants, and get the related [[HoodieCommitMetadata]] objects. + * 2. initialize the [[HoodieTableFileSystemView]] by the touched data files. + * 3. extract the cdc information: + * generate a [[CDCFileSplit]] object for each of the instant in (startInstant, endInstant) + * and each of the file group which is touched in the range of instants. + */ +public class HoodieCDCExtractor { + + private final HoodieTableMetaClient metaClient; + + private final Path basePath; + + private final FileSystem fs; + + private final HoodieCDCSupplementalLoggingMode supplementalLoggingMode; + + private final String startInstant; + + private final String endInstant; + + private Map commits; + + private HoodieTableFileSystemView fsView; + + public HoodieCDCExtractor( + HoodieTableMetaClient metaClient, + String startInstant, + String endInstant) { + this.metaClient = metaClient; + this.basePath = metaClient.getBasePathV2(); + this.fs = metaClient.getFs().getFileSystem(); + this.supplementalLoggingMode = HoodieCDCSupplementalLoggingMode.parse( + metaClient.getTableConfig().cdcSupplementalLoggingMode()); + this.startInstant = startInstant; + this.endInstant = endInstant; + init(); + } + + private void init() { + initInstantAndCommitMetadatas(); + initFSView(); + } + + /** + * At the granularity of a file group, trace the mapping between + * each commit/instant and changes to this file group. + */ + public Map>> extractCDCFileSplits() { + if (commits == null || fsView == null) { + throw new HoodieException("Fail to init CDCExtractor"); + } + + Map>> fgToCommitChanges = new HashMap<>(); + for (HoodieInstant instant : commits.keySet()) { + HoodieCommitMetadata commitMetadata = commits.get(instant); + + // parse `partitionToWriteStats` in the metadata of commit + Map> ptToWriteStats = commitMetadata.getPartitionToWriteStats(); + for (String partition : ptToWriteStats.keySet()) { + List hoodieWriteStats = ptToWriteStats.get(partition); + hoodieWriteStats.forEach(writeStat -> { + HoodieFileGroupId fileGroupId = new HoodieFileGroupId(partition, writeStat.getFileId()); + // Identify the CDC source involved in this commit and + // determine its type for subsequent loading using different methods. + HoodieCDCFileSplit changeFile = + parseWriteStat(fileGroupId, instant, writeStat, commitMetadata.getOperationType()); + fgToCommitChanges.computeIfAbsent(fileGroupId, k -> new ArrayList<>()); + fgToCommitChanges.get(fileGroupId).add(Pair.of(instant, changeFile)); + }); + } + + if (commitMetadata instanceof HoodieReplaceCommitMetadata) { + HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) commitMetadata; + Map> ptToReplacedFileId = replaceCommitMetadata.getPartitionToReplaceFileIds(); + for (String partition : ptToReplacedFileId.keySet()) { + List fileIds = ptToReplacedFileId.get(partition); + fileIds.forEach(fileId -> { + Option latestFileSliceOpt = fsView.fetchLatestFileSlice(partition, fileId); + if (latestFileSliceOpt.isPresent()) { + HoodieFileGroupId fileGroupId = new HoodieFileGroupId(partition, fileId); + HoodieCDCFileSplit changeFile = new HoodieCDCFileSplit( + REPLACE_COMMIT, null, latestFileSliceOpt, Option.empty()); + if (!fgToCommitChanges.containsKey(fileGroupId)) { + fgToCommitChanges.put(fileGroupId, new ArrayList<>()); + } + fgToCommitChanges.get(fileGroupId).add(Pair.of(instant, changeFile)); + } + }); + } + } + } + return fgToCommitChanges; + } + + /** + * Parse the commit metadata between (startInstant, endInstant], and extract the touched partitions + * and files to build the filesystem view. + */ + private void initFSView() { + Set touchedPartitions = new HashSet<>(); + for (Map.Entry entry : commits.entrySet()) { + HoodieCommitMetadata commitMetadata = entry.getValue(); + touchedPartitions.addAll(commitMetadata.getPartitionToWriteStats().keySet()); + if (commitMetadata instanceof HoodieReplaceCommitMetadata) { + touchedPartitions.addAll( + ((HoodieReplaceCommitMetadata) commitMetadata).getPartitionToReplaceFileIds().keySet() + ); + } + } + try { + List touchedFiles = new ArrayList<>(); + for (String touchedPartition : touchedPartitions) { + Path partitionPath = FSUtils.getPartitionPath(basePath, touchedPartition); + touchedFiles.addAll(Arrays.asList(fs.listStatus(partitionPath))); + } + this.fsView = new HoodieTableFileSystemView( + metaClient, + metaClient.getCommitsTimeline().filterCompletedInstants(), + touchedFiles.toArray(new FileStatus[0]) + ); + } catch (Exception e) { + throw new HoodieException("Fail to init FileSystem View for CDC", e); + } + } + + + /** + * Extract the required instants from all the instants between (startInstant, endInstant]. + * + * There are some conditions: + * 1) the instant should be completed; + * 2) the instant should be in (startInstant, endInstant]; + * 3) the action of the instant is one of 'commit', 'deltacommit', 'replacecommit'; + * 4) the write type of the commit should have the ability to change the data. + * + * And, we need to recognize which is a 'replacecommit', that help to find the list of file group replaced. + */ + private void initInstantAndCommitMetadatas() { + try { + List requiredActions = Arrays.asList(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION); + HoodieActiveTimeline activeTimeLine = metaClient.getActiveTimeline(); + Map result = activeTimeLine.getInstants() + .filter(instant -> + instant.isCompleted() + && isInRange(instant.getTimestamp(), startInstant, endInstant) + && requiredActions.contains(instant.getAction().toLowerCase(Locale.ROOT)) + ).map(instant -> { + HoodieCommitMetadata commitMetadata; + try { + if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { + commitMetadata = HoodieReplaceCommitMetadata.fromBytes( + activeTimeLine.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + } else { + commitMetadata = HoodieCommitMetadata.fromBytes( + activeTimeLine.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage()); + } + return Pair.of(instant, commitMetadata); + }).filter(pair -> + maybeChangeData(pair.getRight().getOperationType()) + ).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + this.commits = result; + } catch (Exception e) { + throw new HoodieIOException("Fail to get the commit metadata for CDC"); + } + } + + private Boolean maybeChangeData(WriteOperationType operation) { + return operation == WriteOperationType.INSERT + || operation == WriteOperationType.UPSERT + || operation == WriteOperationType.DELETE + || operation == WriteOperationType.BULK_INSERT + || operation == WriteOperationType.DELETE_PARTITION + || operation == WriteOperationType.INSERT_OVERWRITE + || operation == WriteOperationType.INSERT_OVERWRITE_TABLE + || operation == WriteOperationType.BOOTSTRAP; + } + + /** + * Parse HoodieWriteStat, judge which type the file is, and what strategy should be used to parse CDC data. + * Then build a [[HoodieCDCFileSplit]] object. + */ + private HoodieCDCFileSplit parseWriteStat( + HoodieFileGroupId fileGroupId, + HoodieInstant instant, + HoodieWriteStat writeStat, + WriteOperationType operation) { + Path basePath = metaClient.getBasePathV2(); + FileSystem fs = metaClient.getFs().getFileSystem(); + + HoodieCDCFileSplit cdcFileSplit; + if (StringUtils.isNullOrEmpty(writeStat.getCdcPath())) { + // no cdc log files can be used directly. we reuse the existing data file to retrieve the change data. + String path = writeStat.getPath(); + if (FSUtils.isBaseFile(new Path(path))) { + // this is a base file + if (operation == WriteOperationType.DELETE && writeStat.getNumWrites() == 0L + && writeStat.getNumDeletes() != 0) { + // This is a delete operation wherein all the records in this file group are deleted + // and no records have been writen out a new file. + // So, we find the previous file that this operation delete from, and treat each of + // records as a deleted one. + HoodieBaseFile beforeBaseFile = fsView.getBaseFileOn( + fileGroupId.getPartitionPath(), writeStat.getPrevCommit(), fileGroupId.getFileId() + ).orElseThrow(() -> + new HoodieIOException("Can not get the previous version of the base file") + ); + FileSlice beforeFileSlice = new FileSlice(fileGroupId, writeStat.getPrevCommit(), beforeBaseFile, new ArrayList<>()); + cdcFileSplit = new HoodieCDCFileSplit(BASE_FILE_DELETE, null, Option.empty(), Option.of(beforeFileSlice)); + } else if (writeStat.getNumUpdateWrites() == 0L && writeStat.getNumDeletes() == 0 + && writeStat.getNumWrites() == writeStat.getNumInserts()) { + // all the records in this file are new. + cdcFileSplit = new HoodieCDCFileSplit(BASE_FILE_INSERT, path); + } else { + throw new HoodieException("There should be a cdc log file."); + } + } else { + // this is a log file + Option beforeFileSliceOpt = getDependentFileSliceForLogFile(fileGroupId, instant, path); + cdcFileSplit = new HoodieCDCFileSplit(LOG_FILE, path, beforeFileSliceOpt, Option.empty()); + } + } else { + // this is a cdc log + if (supplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) { + cdcFileSplit = new HoodieCDCFileSplit(AS_IS, writeStat.getCdcPath()); + } else { + try { + HoodieBaseFile beforeBaseFile = fsView.getBaseFileOn( + fileGroupId.getPartitionPath(), writeStat.getPrevCommit(), fileGroupId.getFileId() + ).orElseThrow(() -> + new HoodieIOException("Can not get the previous version of the base file") + ); + FileSlice beforeFileSlice = null; + FileSlice currentFileSlice = new FileSlice(fileGroupId, instant.getTimestamp(), + new HoodieBaseFile(fs.getFileStatus(new Path(basePath, writeStat.getPath()))), new ArrayList<>()); + if (supplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.OP_KEY)) { + beforeFileSlice = new FileSlice(fileGroupId, writeStat.getPrevCommit(), beforeBaseFile, new ArrayList<>()); + } + cdcFileSplit = new HoodieCDCFileSplit(AS_IS, writeStat.getCdcPath(), + Option.ofNullable(beforeFileSlice), Option.ofNullable(currentFileSlice)); + } catch (Exception e) { + throw new HoodieException("Fail to parse HoodieWriteStat", e); + } + } + } + return cdcFileSplit; + } + + /** + * For a mor log file, get the completed previous file slice from the related commit metadata. + * This file slice will be used when we extract the change data from this mor log file. + */ + private Option getDependentFileSliceForLogFile( + HoodieFileGroupId fgId, + HoodieInstant instant, + String currentLogFile) { + Path partitionPath = FSUtils.getPartitionPath(basePath, fgId.getPartitionPath()); + if (instant.getAction().equals(DELTA_COMMIT_ACTION)) { + String currentLogFileName = new Path(currentLogFile).getName(); + Option>> fileSliceOpt = + HoodieCommitMetadata.getFileSliceForFileGroupFromDeltaCommit( + metaClient.getActiveTimeline().getInstantDetails(instant).get(), fgId); + if (fileSliceOpt.isPresent()) { + Pair> fileSlice = fileSliceOpt.get(); + try { + HoodieBaseFile baseFile = new HoodieBaseFile( + fs.getFileStatus(new Path(partitionPath, fileSlice.getLeft()))); + Path[] logFilePaths = fileSlice.getRight().stream() + .filter(logFile -> !logFile.equals(currentLogFileName)) + .map(logFile -> new Path(partitionPath, logFile)) + .toArray(Path[]::new); + List logFiles = Arrays.stream(fs.listStatus(logFilePaths)) + .map(HoodieLogFile::new).collect(Collectors.toList()); + return Option.of(new FileSlice(fgId, instant.getTimestamp(), baseFile, logFiles)); + } catch (Exception e) { + throw new HoodieException("Fail to get the dependent file slice for a log file", e); + } + } else { + return Option.empty(); + } + } + return Option.empty(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java new file mode 100644 index 0000000000000..43ad3bf1761e1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java @@ -0,0 +1,94 @@ +/* + * 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.hudi.common.table.cdc; + +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.util.Option; + +import java.io.Serializable; + +/** + * This contains all the information that retrieve the change data at a single file group and + * at a single commit. + *

+ * For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_INSERT}, `cdcFile` is a current version of + * the base file in the group, and `beforeFileSlice` is None. + * For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_DELETE}, `cdcFile` is null, + * `beforeFileSlice` is the previous version of the base file in the group. + * For `cdcInferCase` = {@link HoodieCDCInferCase#AS_IS}, `cdcFile` is a log file with cdc blocks. + * when enable the supplemental logging, both `beforeFileSlice` and `afterFileSlice` are None, + * otherwise these two are the previous and current version of the base file. + * For `cdcInferCase` = {@link HoodieCDCInferCase#LOG_FILE}, `cdcFile` is a normal log file and + * `beforeFileSlice` is the previous version of the file slice. + * For `cdcInferCase` = {@link HoodieCDCInferCase#REPLACE_COMMIT}, `cdcFile` is null, + * `beforeFileSlice` is the current version of the file slice. + */ +public class HoodieCDCFileSplit implements Serializable { + + /** + * * the change type, which decide to how to retrieve the change data. more details see: `HoodieCDCLogicalFileType#` + */ + private final HoodieCDCInferCase cdcInferCase; + + /** + * the file that the change data can be parsed from. + */ + private final String cdcFile; + + /** + * the file slice that are required when retrieve the before data. + */ + private final Option beforeFileSlice; + + /** + * the file slice that are required when retrieve the after data. + */ + private final Option afterFileSlice; + + public HoodieCDCFileSplit(HoodieCDCInferCase cdcInferCase, String cdcFile) { + this(cdcInferCase, cdcFile, Option.empty(), Option.empty()); + } + + public HoodieCDCFileSplit( + HoodieCDCInferCase cdcInferCase, + String cdcFile, + Option beforeFileSlice, + Option afterFileSlice) { + this.cdcInferCase = cdcInferCase; + this.cdcFile = cdcFile; + this.beforeFileSlice = beforeFileSlice; + this.afterFileSlice = afterFileSlice; + } + + public HoodieCDCInferCase getCdcInferCase() { + return this.cdcInferCase; + } + + public String getCdcFile() { + return this.cdcFile; + } + + public Option getBeforeFileSlice() { + return this.beforeFileSlice; + } + + public Option getAfterFileSlice() { + return this.afterFileSlice; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java new file mode 100644 index 0000000000000..c6005c601012a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java @@ -0,0 +1,78 @@ +/* + * 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.hudi.common.table.cdc; + +/** + * Here define five cdc infer cases. The different cdc infer case will decide which file will be + * used to extract the change data, and how to do this. + * + * AS_IS: + * For this type, there must be a real cdc log file from which we get the whole/part change data. + * when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before_after', it keeps all the fields about the + * change data, including `op`, `ts_ms`, `before` and `after`. So read it and return directly, + * no more other files need to be loaded. + * when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before', it keeps the `op`, the key and the + * `before` of the changing record. When `op` is equal to 'i' or 'u', need to get the current record from the + * current base/log file as `after`. + * when `hoodie.table.cdc.supplemental.logging.mode` is 'op_key', it just keeps the `op` and the key of + * the changing record. When `op` is equal to 'i', `before` is null and get the current record + * from the current base/log file as `after`. When `op` is equal to 'u', get the previous + * record from the previous file slice as `before`, and get the current record from the + * current base/log file as `after`. When `op` is equal to 'd', get the previous record from + * the previous file slice as `before`, and `after` is null. + * + * BASE_FILE_INSERT: + * For this type, there must be a base file at the current instant. All the records from this + * file is new-coming, so we can load this, mark all the records with `i`, and treat them as + * the value of `after`. The value of `before` for each record is null. + * + * BASE_FILE_INSERT: + * For this type, there must be an empty file at the current instant, but a non-empty base file + * at the previous instant. First we find this base file that has the same file group and belongs + * to the previous instant. Then load this, mark all the records with `d`, and treat them as + * the value of `before`. The value of `after` for each record is null. + * + * LOG_FILE: + * For this type, a normal log file of mor table will be used. First we need to load the previous + * file slice(including the base file and other log files in the same file group). Then for each + * record from the log file, get the key of this, and execute the following steps: + * 1) if the record is deleted, + * a) if there is a record with the same key in the data loaded, `op` is 'd', 'before' is the + * record from the data loaded, `after` is null; + * b) if there is not a record with the same key in the data loaded, just skip. + * 2) the record is not deleted, + * a) if there is a record with the same key in the data loaded, `op` is 'u', 'before' is the + * record from the data loaded, `after` is the current record; + * b) if there is not a record with the same key in the data loaded, `op` is 'i', 'before' is + * null, `after` is the current record; + * + * REPLACE_COMMIT: + * For this type, it must be a replacecommit, like INSERT_OVERWRITE and DROP_PARTITION. It drops + * a whole file group. First we find this file group. Then load this, mark all the records with + * `d`, and treat them as the value of `before`. The value of `after` for each record is null. + */ +public enum HoodieCDCInferCase { + + AS_IS, + BASE_FILE_INSERT, + BASE_FILE_DELETE, + LOG_FILE, + REPLACE_COMMIT; + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java new file mode 100644 index 0000000000000..18db9850a0b3f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieCDCLogRecordIterator.java @@ -0,0 +1,78 @@ +/* + * 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.hudi.common.table.log; + +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.log.block.HoodieDataBlock; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +public class HoodieCDCLogRecordIterator implements ClosableIterator { + + private final HoodieLogFile cdcLogFile; + + private final HoodieLogFormat.Reader reader; + + private ClosableIterator itr; + + public HoodieCDCLogRecordIterator( + FileSystem fs, + Path cdcLogPath, + Schema cdcSchema) throws IOException { + this.cdcLogFile = new HoodieLogFile(fs.getFileStatus(cdcLogPath)); + this.reader = new HoodieLogFileReader(fs, cdcLogFile, cdcSchema, + HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false); + } + + @Override + public boolean hasNext() { + if (itr == null || !itr.hasNext()) { + if (reader.hasNext()) { + HoodieDataBlock dataBlock = (HoodieDataBlock) reader.next(); + itr = dataBlock.getRecordIterator(); + return itr.hasNext(); + } + return false; + } + return true; + } + + @Override + public IndexedRecord next() { + return itr.next(); + } + + @Override + public void close() { + try { + itr.close(); + reader.close(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage()); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 89a184bf497f5..d70f74a0c454a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -1109,7 +1109,7 @@ protected Option fetchLatestBaseFile(String partitionPath, Strin * @param fileId File Id * @return File Slice if present */ - protected Option fetchLatestFileSlice(String partitionPath, String fileId) { + public Option fetchLatestFileSlice(String partitionPath, String fileId) { return Option .fromJavaOptional(fetchLatestFileSlices(partitionPath).filter(fs -> fs.getFileId().equals(fileId)).findFirst()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index 02a406e7e0763..ebd759f88060b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -399,7 +399,7 @@ Stream fetchAllStoredFileGroups() { } @Override - protected Option fetchLatestFileSlice(String partitionPath, String fileId) { + public Option fetchLatestFileSlice(String partitionPath, String fileId) { // Retries only file-slices of the file and filters for the latest return Option.ofNullable(rocksDB .prefixSearch(schemaHelper.getColFamilyForView(), diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index e8ffb09ff9100..1dd339b975cfe 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -61,6 +61,16 @@ object DataSourceReadOptions { "(or) Read Optimized mode (obtain latest view, based on base files) (or) Snapshot mode " + "(obtain latest view, by merging base and (if any) log files)") + val INCREMENTAL_FORMAT_LATEST_STATE_VAL = "latest_state" + val INCREMENTAL_FORMAT_CDC_VAL = "cdc" + val INCREMENTAL_FORMAT: ConfigProperty[String] = ConfigProperty + .key("hoodie.datasource.query.incremental.format") + .defaultValue(INCREMENTAL_FORMAT_LATEST_STATE_VAL) + .withValidValues(INCREMENTAL_FORMAT_LATEST_STATE_VAL, INCREMENTAL_FORMAT_CDC_VAL) + .withDocumentation("This config is used alone with the 'incremental' query type." + + "When set to 'latest_state', it returns the latest records' values." + + "When set to 'cdc', it returns the cdc data.") + val REALTIME_SKIP_MERGE_OPT_VAL = "skip_merge" val REALTIME_PAYLOAD_COMBINE_OPT_VAL = "payload_combine" val REALTIME_MERGE: ConfigProperty[String] = ConfigProperty diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index c7608c98291be..cd3fe6832d757 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -18,8 +18,10 @@ package org.apache.hudi import org.apache.hadoop.fs.Path + import org.apache.hudi.DataSourceReadOptions._ import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION} +import org.apache.hudi.cdc.CDCRelation import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} @@ -27,7 +29,9 @@ import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException import org.apache.hudi.util.PathUtils + import org.apache.log4j.LogManager + import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog import org.apache.spark.sql.hudi.streaming.HoodieStreamSource @@ -171,16 +175,8 @@ class DefaultSource extends RelationProvider } val metaClient = HoodieTableMetaClient.builder().setConf( sqlContext.sparkSession.sessionState.newHadoopConf()).setBasePath(path.get).build() - val schemaResolver = new TableSchemaResolver(metaClient) - val sqlSchema = - try { - val avroSchema = schemaResolver.getTableAvroSchema - AvroConversionUtils.convertAvroSchemaToStructType(avroSchema) - } catch { - case _: Exception => - require(schema.isDefined, "Fail to resolve source schema") - schema.get - } + + val sqlSchema = DefaultSource.resolveSchema(metaClient, parameters, schema) (shortName(), sqlSchema) } @@ -205,6 +201,8 @@ object DefaultSource { val tableType = metaClient.getTableType val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent val queryType = parameters(QUERY_TYPE.key) + val isCdcQuery = queryType == QUERY_TYPE_INCREMENTAL_OPT_VAL && + parameters.get(INCREMENTAL_FORMAT.key).contains(INCREMENTAL_FORMAT_CDC_VAL) log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType") @@ -219,7 +217,9 @@ object DefaultSource { } if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { - new EmptyRelation(sqlContext, metaClient) + new EmptyRelation(sqlContext, resolveSchema(metaClient, parameters, Some(schema))) + } else if (isCdcQuery) { + CDCRelation.getCDCRelation(sqlContext, metaClient, parameters) } else { (tableType, queryType, isBootstrappedTable) match { case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) | @@ -264,4 +264,25 @@ object DefaultSource { baseRelation.toHadoopFsRelation } } + + private def resolveSchema(metaClient: HoodieTableMetaClient, + parameters: Map[String, String], + schema: Option[StructType]): StructType = { + val isCdcQuery = CDCRelation.isCDCEnabled(metaClient) && + parameters.get(QUERY_TYPE.key).contains(QUERY_TYPE_INCREMENTAL_OPT_VAL) && + parameters.get(INCREMENTAL_FORMAT.key).contains(INCREMENTAL_FORMAT_CDC_VAL) + if (isCdcQuery) { + CDCRelation.FULL_CDC_SPARK_SCHEMA + } else { + val schemaResolver = new TableSchemaResolver(metaClient) + try { + val avroSchema = schemaResolver.getTableAvroSchema + AvroConversionUtils.convertAvroSchemaToStructType(avroSchema) + } catch { + case _: Exception => + require(schema.isDefined, "Fail to resolve source schema") + schema.get + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/EmptyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/EmptyRelation.scala index 3645eb8d9b9e8..b553bb28bd125 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/EmptyRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/EmptyRelation.scala @@ -19,31 +19,17 @@ package org.apache.hudi -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SQLContext} -import scala.util.control.NonFatal - /** * BaseRelation representing empty RDD. * @param sqlContext instance of SqlContext. */ -class EmptyRelation(val sqlContext: SQLContext, metaClient: HoodieTableMetaClient) extends BaseRelation with TableScan { - - override def schema: StructType = { - // do the best to find the table schema. - val schemaResolver = new TableSchemaResolver(metaClient) - try { - val avroSchema = schemaResolver.getTableAvroSchema - AvroConversionUtils.convertAvroSchemaToStructType(avroSchema) - } catch { - case NonFatal(e) => - StructType(Nil) - } - } +class EmptyRelation(val sqlContext: SQLContext, + override val schema: StructType) extends BaseRelation with TableScan { override def buildScan(): RDD[Row] = { sqlContext.sparkContext.emptyRDD[Row] diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala index 8bd295c7f3db4..47c7b6efece7c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala @@ -18,13 +18,17 @@ package org.apache.hudi +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileStatus import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.util.StringUtils.isNullOrEmpty +import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{PredicateHelper, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.datasources.PartitionedFile @@ -84,4 +88,17 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport { PartitionedFile(partitionValues, filePath.toUri.toString, offset, size) } } + + trait AvroDeserializerSupport extends SparkAdapterSupport { + protected val avroSchema: Schema + protected val structTypeSchema: StructType + + private lazy val deserializer: HoodieAvroDeserializer = + sparkAdapter.createAvroDeserializer(avroSchema, structTypeSchema) + + protected def deserialize(avroRecord: GenericRecord): InternalRow = { + checkState(avroRecord.getSchema.getFields.size() == structTypeSchema.fields.length) + deserializer.deserialize(avroRecord).get.asInstanceOf[InternalRow] + } + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 512c97806f31d..f122c9f9d673d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -18,40 +18,20 @@ package org.apache.hudi -import org.apache.avro.Schema -import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.JobConf -import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, generateUnsafeProjection, projectReader} -import org.apache.hudi.HoodieConversionUtils.{toJavaOption, toScalaOption} -import org.apache.hudi.HoodieMergeOnReadRDD.SafeAvroProjection.collectFieldOrdinals -import org.apache.hudi.HoodieMergeOnReadRDD._ -import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig} -import org.apache.hudi.common.engine.HoodieLocalEngineContext -import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath -import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord, HoodieRecordPayload, OverwriteWithLatestAvroPayload} -import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner -import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.config.HoodiePayloadConfig + +import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, projectReader} +import org.apache.hudi.LogFileIterator.CONFIG_INSTANTIATION_LOCK +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload import org.apache.hudi.exception.HoodieException -import org.apache.hudi.hadoop.config.HoodieRealtimeConfig import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes -import org.apache.hudi.internal.schema.InternalSchema -import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable -import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} + import org.apache.spark.rdd.RDD -import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.types.StructType import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext} import java.io.Closeable -import java.util.Properties -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.util.Try case class HoodieMergeOnReadPartition(index: Int, split: HoodieMergeOnReadFileSplit) extends Partition @@ -100,14 +80,6 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, protected val maxCompactionMemoryInBytes: Long = getMaxCompactionMemoryInBytes(new JobConf(config)) private val confBroadcast = sc.broadcast(new SerializableWritable(config)) - private val payloadProps = tableState.preCombineFieldOpt - .map(preCombineField => - HoodiePayloadConfig.newBuilder - .withPayloadOrderingField(preCombineField) - .build - .getProps - ) - .getOrElse(new Properties()) private val whitelistedPayloadClasses: Set[String] = Seq( classOf[OverwriteWithLatestAvroPayload] @@ -121,15 +93,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, projectedReader(dataFileOnlySplit.dataFile.get) case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => - new LogFileIterator(logFileOnlySplit, getConfig) + new LogFileIterator(logFileOnlySplit, tableSchema, requiredSchema, tableState, getConfig) case split if mergeType.equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) => val reader = fileReaders.requiredSchemaReaderSkipMerging - new SkipMergeIterator(split, reader, getConfig) + new SkipMergeIterator(split, reader, tableSchema, requiredSchema, tableState, getConfig) case split if mergeType.equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) => val reader = pickBaseFileReader - new RecordMergingFileIterator(split, reader, getConfig) + new RecordMergingFileIterator(split, reader, tableSchema, requiredSchema, tableState, getConfig) case _ => throw new HoodieException(s"Unable to select an Iterator to read the Hoodie MOR File Split for " + s"file path: ${mergeOnReadPartition.split.dataFile.get.filePath}" + @@ -169,310 +141,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private def getConfig: Configuration = { val conf = confBroadcast.value.value - HoodieMergeOnReadRDD.CONFIG_INSTANTIATION_LOCK.synchronized { + CONFIG_INSTANTIATION_LOCK.synchronized { new Configuration(conf) } } - - /** - * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], iterates over all of the records stored in - * Delta Log files (represented as [[InternalRow]]s) - */ - private class LogFileIterator(split: HoodieMergeOnReadFileSplit, - config: Configuration) - extends Iterator[InternalRow] with Closeable with AvroDeserializerSupport { - - protected override val requiredAvroSchema: Schema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) - protected override val requiredStructTypeSchema: StructType = requiredSchema.structTypeSchema - - protected val logFileReaderAvroSchema: Schema = new Schema.Parser().parse(tableSchema.avroSchemaStr) - - protected var recordToLoad: InternalRow = _ - - private val requiredSchemaSafeAvroProjection = SafeAvroProjection.create(logFileReaderAvroSchema, requiredAvroSchema) - - private var logScanner = { - val internalSchema = tableSchema.internalSchema.getOrElse(InternalSchema.getEmptyInternalSchema) - HoodieMergeOnReadRDD.scanLog(split.logFiles, getPartitionPath(split), logFileReaderAvroSchema, tableState, - maxCompactionMemoryInBytes, config, internalSchema) - } - - private val logRecords = logScanner.getRecords.asScala - - // NOTE: This have to stay lazy to make sure it's initialized only at the point where it's - // going to be used, since we modify `logRecords` before that and therefore can't do it any earlier - protected lazy val logRecordsIterator: Iterator[Option[GenericRecord]] = - logRecords.iterator.map { - case (_, record) => - toScalaOption(record.getData.getInsertValue(logFileReaderAvroSchema, payloadProps)) - .map(_.asInstanceOf[GenericRecord]) - } - - protected def removeLogRecord(key: String): Option[HoodieRecord[_ <: HoodieRecordPayload[_]]] = - logRecords.remove(key) - - override def hasNext: Boolean = hasNextInternal - - // NOTE: It's crucial for this method to be annotated w/ [[@tailrec]] to make sure - // that recursion is unfolded into a loop to avoid stack overflows while - // handling records - @tailrec private def hasNextInternal: Boolean = { - logRecordsIterator.hasNext && { - val avroRecordOpt = logRecordsIterator.next() - if (avroRecordOpt.isEmpty) { - // Record has been deleted, skipping - this.hasNextInternal - } else { - val projectedAvroRecord = requiredSchemaSafeAvroProjection(avroRecordOpt.get) - recordToLoad = deserialize(projectedAvroRecord) - true - } - } - } - - override final def next(): InternalRow = recordToLoad - - override def close(): Unit = - if (logScanner != null) { - try { - logScanner.close() - } finally { - logScanner = null - } - } - } - - /** - * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], provides an iterator over all of the records stored in - * Base file as well as all of the Delta Log files simply returning concatenation of these streams, while not - * performing any combination/merging of the records w/ the same primary keys (ie producing duplicates potentially) - */ - private class SkipMergeIterator(split: HoodieMergeOnReadFileSplit, - baseFileReader: BaseFileReader, - config: Configuration) - extends LogFileIterator(split, config) { - - private val requiredSchemaUnsafeProjection = generateUnsafeProjection(baseFileReader.schema, requiredStructTypeSchema) - - private val baseFileIterator = baseFileReader(split.dataFile.get) - - override def hasNext: Boolean = { - if (baseFileIterator.hasNext) { - // No merge is required, simply load current row and project into required schema - recordToLoad = requiredSchemaUnsafeProjection(baseFileIterator.next()) - true - } else { - super[LogFileIterator].hasNext - } - } - } - - /** - * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], provides an iterator over all of the records stored in - * a) Base file and all of the b) Delta Log files combining records with the same primary key from both of these - * streams - */ - private class RecordMergingFileIterator(split: HoodieMergeOnReadFileSplit, - baseFileReader: BaseFileReader, - config: Configuration) - extends LogFileIterator(split, config) { - - // NOTE: Record-merging iterator supports 2 modes of operation merging records bearing either - // - Full table's schema - // - Projected schema - // As such, no particular schema could be assumed, and therefore we rely on the caller - // to correspondingly set the scheme of the expected output of base-file reader - private val baseFileReaderAvroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(baseFileReader.schema, nullable = false, "record") - - private val serializer = sparkAdapter.createAvroSerializer(baseFileReader.schema, baseFileReaderAvroSchema, nullable = false) - - private val reusableRecordBuilder: GenericRecordBuilder = new GenericRecordBuilder(requiredAvroSchema) - - private val recordKeyOrdinal = baseFileReader.schema.fieldIndex(tableState.recordKeyField) - - private val requiredSchemaUnsafeProjection = generateUnsafeProjection(baseFileReader.schema, requiredStructTypeSchema) - - private val baseFileIterator = baseFileReader(split.dataFile.get) - - override def hasNext: Boolean = hasNextInternal - - // NOTE: It's crucial for this method to be annotated w/ [[@tailrec]] to make sure - // that recursion is unfolded into a loop to avoid stack overflows while - // handling records - @tailrec private def hasNextInternal: Boolean = { - if (baseFileIterator.hasNext) { - val curRow = baseFileIterator.next() - val curKey = curRow.getString(recordKeyOrdinal) - val updatedRecordOpt = removeLogRecord(curKey) - if (updatedRecordOpt.isEmpty) { - // No merge is required, simply load current row and project into required schema - recordToLoad = requiredSchemaUnsafeProjection(curRow) - true - } else { - val mergedAvroRecordOpt = merge(serialize(curRow), updatedRecordOpt.get) - if (mergedAvroRecordOpt.isEmpty) { - // Record has been deleted, skipping - this.hasNextInternal - } else { - val projectedAvroRecord = projectAvroUnsafe(mergedAvroRecordOpt.get.asInstanceOf[GenericRecord], - requiredAvroSchema, reusableRecordBuilder) - recordToLoad = deserialize(projectedAvroRecord) - true - } - } - } else { - super[LogFileIterator].hasNext - } - } - - private def serialize(curRowRecord: InternalRow): GenericRecord = - serializer.serialize(curRowRecord).asInstanceOf[GenericRecord] - - private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): Option[IndexedRecord] = { - // NOTE: We have to pass in Avro Schema used to read from Delta Log file since we invoke combining API - // on the record from the Delta Log - toScalaOption(newRecord.getData.combineAndGetUpdateValue(curAvroRecord, logFileReaderAvroSchema, payloadProps)) - } - } -} - -private object HoodieMergeOnReadRDD { - - val CONFIG_INSTANTIATION_LOCK = new Object() - - def scanLog(logFiles: List[HoodieLogFile], - partitionPath: Path, - logSchema: Schema, - tableState: HoodieTableState, - maxCompactionMemoryInBytes: Long, - hadoopConf: Configuration, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema): HoodieMergedLogRecordScanner = { - val tablePath = tableState.tablePath - val fs = FSUtils.getFs(tablePath, hadoopConf) - - if (HoodieTableMetadata.isMetadataTable(tablePath)) { - val metadataConfig = HoodieMetadataConfig.newBuilder() - .fromProperties(tableState.metadataConfig.getProps).enable(true).build() - val dataTableBasePath = getDataTableBasePathFromMetadataTable(tablePath) - val metadataTable = new HoodieBackedTableMetadata( - new HoodieLocalEngineContext(hadoopConf), metadataConfig, - dataTableBasePath, - hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) - - // We have to force full-scan for the MT log record reader, to make sure - // we can iterate over all of the partitions, since by default some of the partitions (Column Stats, - // Bloom Filter) are in "point-lookup" mode - val forceFullScan = true - - // NOTE: In case of Metadata Table partition path equates to partition name (since there's just one level - // of indirection among MT partitions) - val relativePartitionPath = getRelativePartitionPath(new Path(tablePath), partitionPath) - metadataTable.getLogRecordScanner(logFiles.asJava, relativePartitionPath, toJavaOption(Some(forceFullScan))) - .getLeft - } else { - val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(tablePath) - .withLogFilePaths(logFiles.map(logFile => logFile.getPath.toString).asJava) - .withReaderSchema(logSchema) - .withLatestInstantTime(tableState.latestCommitTimestamp) - .withReadBlocksLazily( - Try(hadoopConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, - HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean) - .getOrElse(false)) - .withReverseReader(false) - .withInternalSchema(internalSchema) - .withBufferSize( - hadoopConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, - HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) - .withMaxMemorySizeInBytes(maxCompactionMemoryInBytes) - .withSpillableMapBasePath( - hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, - HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) - .withDiskMapType( - hadoopConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key, - HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue)) - .withBitCaskDiskMapCompressionEnabled( - hadoopConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), - HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())) - if (logFiles.nonEmpty) { - logRecordScannerBuilder.withPartition( - getRelativePartitionPath(new Path(tableState.tablePath), logFiles.head.getPath.getParent)) - } - - logRecordScannerBuilder.build() - } - } - - private def projectAvroUnsafe(record: GenericRecord, projectedSchema: Schema, reusableRecordBuilder: GenericRecordBuilder): GenericRecord = { - val fields = projectedSchema.getFields.asScala - fields.foreach(field => reusableRecordBuilder.set(field, record.get(field.name()))) - reusableRecordBuilder.build() - } - - private def getPartitionPath(split: HoodieMergeOnReadFileSplit): Path = { - // Determine partition path as an immediate parent folder of either - // - The base file - // - Some log file - split.dataFile.map(baseFile => new Path(baseFile.filePath)) - .getOrElse(split.logFiles.head.getPath) - .getParent - } - - // TODO extract to HoodieAvroSchemaUtils - abstract class AvroProjection extends (GenericRecord => GenericRecord) - - class SafeAvroProjection(sourceSchema: Schema, - projectedSchema: Schema, - reusableRecordBuilder: GenericRecordBuilder = null) extends AvroProjection { - - private val ordinals: List[Int] = collectFieldOrdinals(projectedSchema, sourceSchema) - private val recordBuilder: GenericRecordBuilder = - if (reusableRecordBuilder != null) { - reusableRecordBuilder - } else { - new GenericRecordBuilder(projectedSchema) - } - - override def apply(record: GenericRecord): GenericRecord = { - val fields = projectedSchema.getFields.asScala - checkState(fields.length == ordinals.length) - fields.zip(ordinals).foreach { - case (field, pos) => recordBuilder.set(field, record.get(pos)) - } - recordBuilder.build() - } - } - - object SafeAvroProjection { - def create(sourceSchema: Schema, projectedSchema: Schema, reusableRecordBuilder: GenericRecordBuilder = null): SafeAvroProjection = - new SafeAvroProjection( - sourceSchema = sourceSchema, - projectedSchema = projectedSchema, - reusableRecordBuilder = reusableRecordBuilder) - - /** - * Maps [[projected]] [[Schema]] onto [[source]] one, collecting corresponding field ordinals w/in it, which - * will be subsequently used by either [[projectRowUnsafe]] or [[projectAvroUnsafe()]] method - * - * @param projected target projected schema (which is a proper subset of [[source]] [[Schema]]) - * @param source source schema of the record being projected - * @return list of ordinals of corresponding fields of [[projected]] schema w/in [[source]] one - */ - private def collectFieldOrdinals(projected: Schema, source: Schema): List[Int] = { - projected.getFields.asScala.map(f => source.getField(f.name()).pos()).toList - } - } - - trait AvroDeserializerSupport extends SparkAdapterSupport { - protected val requiredAvroSchema: Schema - protected val requiredStructTypeSchema: StructType - - private lazy val deserializer: HoodieAvroDeserializer = - sparkAdapter.createAvroDeserializer(requiredAvroSchema, requiredStructTypeSchema) - - protected def deserialize(avroRecord: GenericRecord): InternalRow = { - checkState(avroRecord.getSchema.getFields.size() == requiredStructTypeSchema.fields.length) - deserializer.deserialize(avroRecord).get.asInstanceOf[InternalRow] - } - } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala new file mode 100644 index 0000000000000..07a0ce7f239be --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/LogFileIterator.scala @@ -0,0 +1,323 @@ +/* + * 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.hudi + +import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, generateUnsafeProjection} +import org.apache.hudi.HoodieConversionUtils.{toJavaOption, toScalaOption} +import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport +import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord, HoodieRecordPayload} +import org.apache.hudi.config.HoodiePayloadConfig +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes +import org.apache.hudi.LogFileIterator._ +import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMetadataConfig} +import org.apache.hudi.common.engine.HoodieLocalEngineContext +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner +import org.apache.hudi.hadoop.config.HoodieRealtimeConfig +import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} +import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable + +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.JobConf + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType + +import java.io.Closeable +import java.util.Properties + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.util.Try + +/** + * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], iterates over all of the records stored in + * Delta Log files (represented as [[InternalRow]]s) + */ +class LogFileIterator(split: HoodieMergeOnReadFileSplit, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + tableState: HoodieTableState, + config: Configuration) + extends Iterator[InternalRow] with Closeable with AvroDeserializerSupport { + + protected val maxCompactionMemoryInBytes: Long = getMaxCompactionMemoryInBytes(new JobConf(config)) + + protected val payloadProps = tableState.preCombineFieldOpt + .map { preCombineField => + HoodiePayloadConfig.newBuilder + .withPayloadOrderingField(preCombineField) + .build + .getProps + }.getOrElse(new Properties()) + + protected override val avroSchema: Schema = new Schema.Parser().parse(requiredSchema.avroSchemaStr) + protected override val structTypeSchema: StructType = requiredSchema.structTypeSchema + + protected val logFileReaderAvroSchema: Schema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + + protected var recordToLoad: InternalRow = _ + + private val requiredSchemaSafeAvroProjection = SafeAvroProjection.create(logFileReaderAvroSchema, avroSchema) + + // TODO: now logScanner with internalSchema support column project, we may no need projectAvroUnsafe + private var logScanner = { + val internalSchema = tableSchema.internalSchema.getOrElse(InternalSchema.getEmptyInternalSchema) + scanLog(split.logFiles, getPartitionPath(split), logFileReaderAvroSchema, tableState, + maxCompactionMemoryInBytes, config, internalSchema) + } + + private val logRecords = logScanner.getRecords.asScala + + def logRecordsIterator(): Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])] = { + logRecords.iterator.asInstanceOf[Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])]] + } + + // NOTE: This have to stay lazy to make sure it's initialized only at the point where it's + // going to be used, since we modify `logRecords` before that and therefore can't do it any earlier + protected lazy val genericRecordsIterator: Iterator[Option[GenericRecord]] = + logRecords.iterator.map { + case (_, record) => + toScalaOption(record.getData.getInsertValue(logFileReaderAvroSchema, payloadProps)) + .map(_.asInstanceOf[GenericRecord]) + } + + protected def removeLogRecord(key: String): Option[HoodieRecord[_ <: HoodieRecordPayload[_]]] = + logRecords.remove(key) + + override def hasNext: Boolean = hasNextInternal + + // NOTE: It's crucial for this method to be annotated w/ [[@tailrec]] to make sure + // that recursion is unfolded into a loop to avoid stack overflows while + // handling records + @tailrec private def hasNextInternal: Boolean = { + genericRecordsIterator.hasNext && { + val avroRecordOpt = genericRecordsIterator.next() + if (avroRecordOpt.isEmpty) { + // Record has been deleted, skipping + this.hasNextInternal + } else { + val projectedAvroRecord = requiredSchemaSafeAvroProjection(avroRecordOpt.get) + recordToLoad = deserialize(projectedAvroRecord) + true + } + } + } + + override final def next(): InternalRow = recordToLoad + + override def close(): Unit = + if (logScanner != null) { + try { + logScanner.close() + } finally { + logScanner = null + } + } +} + +/** + * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], provides an iterator over all of the records stored in + * Base file as well as all of the Delta Log files simply returning concatenation of these streams, while not + * performing any combination/merging of the records w/ the same primary keys (ie producing duplicates potentially) + */ +private class SkipMergeIterator(split: HoodieMergeOnReadFileSplit, + baseFileReader: BaseFileReader, + dataSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + tableState: HoodieTableState, + config: Configuration) + extends LogFileIterator(split, dataSchema, requiredSchema, tableState, config) { + + private val requiredSchemaUnsafeProjection = generateUnsafeProjection(baseFileReader.schema, structTypeSchema) + + private val baseFileIterator = baseFileReader(split.dataFile.get) + + override def hasNext: Boolean = { + if (baseFileIterator.hasNext) { + // No merge is required, simply load current row and project into required schema + recordToLoad = requiredSchemaUnsafeProjection(baseFileIterator.next()) + true + } else { + super[LogFileIterator].hasNext + } + } +} + +/** + * Provided w/ instance of [[HoodieMergeOnReadFileSplit]], provides an iterator over all of the records stored in + * a) Base file and all of the b) Delta Log files combining records with the same primary key from both of these + * streams + */ +class RecordMergingFileIterator(split: HoodieMergeOnReadFileSplit, + baseFileReader: BaseFileReader, + dataSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + tableState: HoodieTableState, + config: Configuration) + extends LogFileIterator(split, dataSchema, requiredSchema, tableState, config) { + + // NOTE: Record-merging iterator supports 2 modes of operation merging records bearing either + // - Full table's schema + // - Projected schema + // As such, no particular schema could be assumed, and therefore we rely on the caller + // to correspondingly set the scheme of the expected output of base-file reader + private val baseFileReaderAvroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(baseFileReader.schema, nullable = false, "record") + + private val serializer = sparkAdapter.createAvroSerializer(baseFileReader.schema, baseFileReaderAvroSchema, nullable = false) + + private val reusableRecordBuilder: GenericRecordBuilder = new GenericRecordBuilder(avroSchema) + + private val recordKeyOrdinal = baseFileReader.schema.fieldIndex(tableState.recordKeyField) + + private val requiredSchemaUnsafeProjection = generateUnsafeProjection(baseFileReader.schema, structTypeSchema) + + private val baseFileIterator = baseFileReader(split.dataFile.get) + + override def hasNext: Boolean = hasNextInternal + + // NOTE: It's crucial for this method to be annotated w/ [[@tailrec]] to make sure + // that recursion is unfolded into a loop to avoid stack overflows while + // handling records + @tailrec private def hasNextInternal: Boolean = { + if (baseFileIterator.hasNext) { + val curRow = baseFileIterator.next() + val curKey = curRow.getString(recordKeyOrdinal) + val updatedRecordOpt = removeLogRecord(curKey) + if (updatedRecordOpt.isEmpty) { + // No merge is required, simply load current row and project into required schema + recordToLoad = requiredSchemaUnsafeProjection(curRow) + true + } else { + val mergedAvroRecordOpt = merge(serialize(curRow), updatedRecordOpt.get) + if (mergedAvroRecordOpt.isEmpty) { + // Record has been deleted, skipping + this.hasNextInternal + } else { + val projectedAvroRecord = projectAvroUnsafe(mergedAvroRecordOpt.get.asInstanceOf[GenericRecord], + avroSchema, reusableRecordBuilder) + recordToLoad = deserialize(projectedAvroRecord) + true + } + } + } else { + super[LogFileIterator].hasNext + } + } + + private def serialize(curRowRecord: InternalRow): GenericRecord = + serializer.serialize(curRowRecord).asInstanceOf[GenericRecord] + + private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): Option[IndexedRecord] = { + // NOTE: We have to pass in Avro Schema used to read from Delta Log file since we invoke combining API + // on the record from the Delta Log + toScalaOption(newRecord.getData.combineAndGetUpdateValue(curAvroRecord, logFileReaderAvroSchema, payloadProps)) + } +} + +object LogFileIterator { + + val CONFIG_INSTANTIATION_LOCK = new Object() + + def scanLog(logFiles: List[HoodieLogFile], + partitionPath: Path, + logSchema: Schema, + tableState: HoodieTableState, + maxCompactionMemoryInBytes: Long, + hadoopConf: Configuration, + internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema): HoodieMergedLogRecordScanner = { + val tablePath = tableState.tablePath + val fs = FSUtils.getFs(tablePath, hadoopConf) + + if (HoodieTableMetadata.isMetadataTable(tablePath)) { + val metadataConfig = HoodieMetadataConfig.newBuilder() + .fromProperties(tableState.metadataConfig.getProps).enable(true).build() + val dataTableBasePath = getDataTableBasePathFromMetadataTable(tablePath) + val metadataTable = new HoodieBackedTableMetadata( + new HoodieLocalEngineContext(hadoopConf), metadataConfig, + dataTableBasePath, + hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + + // We have to force full-scan for the MT log record reader, to make sure + // we can iterate over all of the partitions, since by default some of the partitions (Column Stats, + // Bloom Filter) are in "point-lookup" mode + val forceFullScan = true + + // NOTE: In case of Metadata Table partition path equates to partition name (since there's just one level + // of indirection among MT partitions) + val relativePartitionPath = getRelativePartitionPath(new Path(tablePath), partitionPath) + metadataTable.getLogRecordScanner(logFiles.asJava, relativePartitionPath, toJavaOption(Some(forceFullScan))) + .getLeft + } else { + val logRecordScannerBuilder = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(tablePath) + .withLogFilePaths(logFiles.map(logFile => logFile.getPath.toString).asJava) + .withReaderSchema(logSchema) + .withLatestInstantTime(tableState.latestCommitTimestamp) + .withReadBlocksLazily( + Try(hadoopConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean) + .getOrElse(false)) + .withReverseReader(false) + .withInternalSchema(internalSchema) + .withBufferSize( + hadoopConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, + HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withMaxMemorySizeInBytes(maxCompactionMemoryInBytes) + .withSpillableMapBasePath( + hadoopConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, + HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + .withDiskMapType( + hadoopConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key, + HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue)) + .withBitCaskDiskMapCompressionEnabled( + hadoopConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), + HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())) + + if (logFiles.nonEmpty) { + logRecordScannerBuilder.withPartition( + getRelativePartitionPath(new Path(tableState.tablePath), logFiles.head.getPath.getParent)) + } + + logRecordScannerBuilder.build() + } + } + + def projectAvroUnsafe(record: GenericRecord, projectedSchema: Schema, reusableRecordBuilder: GenericRecordBuilder): GenericRecord = { + val fields = projectedSchema.getFields.asScala + fields.foreach(field => reusableRecordBuilder.set(field, record.get(field.name()))) + reusableRecordBuilder.build() + } + + def getPartitionPath(split: HoodieMergeOnReadFileSplit): Path = { + // Determine partition path as an immediate parent folder of either + // - The base file + // - Some log file + split.dataFile.map(baseFile => new Path(baseFile.filePath)) + .getOrElse(split.logFiles.head.getPath) + .getParent + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SafeAvroProjection.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SafeAvroProjection.scala new file mode 100644 index 0000000000000..90978069177f7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SafeAvroProjection.scala @@ -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.hudi + +import org.apache.hudi.SafeAvroProjection.collectFieldOrdinals +import org.apache.hudi.common.util.ValidationUtils.checkState + +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder} + +import scala.collection.JavaConverters._ + +// TODO extract to HoodieAvroSchemaUtils +abstract class AvroProjection extends (GenericRecord => GenericRecord) + +class SafeAvroProjection(sourceSchema: Schema, + projectedSchema: Schema, + reusableRecordBuilder: GenericRecordBuilder = null) extends AvroProjection { + + private val ordinals: List[Int] = collectFieldOrdinals(projectedSchema, sourceSchema) + private val recordBuilder: GenericRecordBuilder = + if (reusableRecordBuilder != null) { + reusableRecordBuilder + } else { + new GenericRecordBuilder(projectedSchema) + } + + override def apply(record: GenericRecord): GenericRecord = { + val fields = projectedSchema.getFields.asScala + checkState(fields.length == ordinals.length) + fields.zip(ordinals).foreach { + case (field, pos) => recordBuilder.set(field, record.get(pos)) + } + recordBuilder.build() + } +} + +object SafeAvroProjection { + def create(sourceSchema: Schema, projectedSchema: Schema, reusableRecordBuilder: GenericRecordBuilder = null): SafeAvroProjection = + new SafeAvroProjection( + sourceSchema = sourceSchema, + projectedSchema = projectedSchema, + reusableRecordBuilder = reusableRecordBuilder) + + /** + * Maps [[projected]] [[Schema]] onto [[source]] one, collecting corresponding field ordinals w/in it, which + * will be subsequently used by either [[projectRowUnsafe]] or [[projectAvroUnsafe()]] method + * + * @param projected target projected schema (which is a proper subset of [[source]] [[Schema]]) + * @param source source schema of the record being projected + * @return list of ordinals of corresponding fields of [[projected]] schema w/in [[source]] one + */ + private def collectFieldOrdinals(projected: Schema, source: Schema): List[Int] = { + projected.getFields.asScala.map(f => source.getField(f.name()).pos()).toList + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala new file mode 100644 index 0000000000000..c4c8f1036014f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala @@ -0,0 +1,203 @@ +/* + * 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.hudi.cdc + +import org.apache.hudi.AvroConversionUtils +import org.apache.hudi.DataSourceReadOptions +import org.apache.hudi.HoodieDataSourceHelper +import org.apache.hudi.HoodieTableSchema +import org.apache.hudi.common.table.cdc.HoodieCDCUtils._ +import org.apache.hudi.common.table.cdc.HoodieCDCOperation._ +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.TableSchemaResolver +import org.apache.hudi.common.table.cdc.HoodieCDCExtractor +import org.apache.hudi.exception.HoodieException +import org.apache.hudi.internal.schema.InternalSchema + +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.{Row, SQLContext, SparkSession} +import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +/** + * Hoodie CDC Relation extends Spark's [[BaseRelation]], provide the schema of cdc + * and the [[buildScan]] to return the change-data in a specified range. + */ +class CDCRelation( + override val sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, + startInstant: String, + endInstant: String, + options: Map[String, String] +) extends BaseRelation with PrunedFilteredScan with Logging { + + val spark: SparkSession = sqlContext.sparkSession + + val (tableAvroSchema, _) = { + val schemaUtil = new TableSchemaResolver(metaClient) + val avroSchema = Try(schemaUtil.getTableAvroSchema) match { + case Success(schema) => schema + case Failure(e) => + throw new IllegalArgumentException("Failed to fetch schema from the table", e) + } + // try to find internalSchema + val internalSchemaFromMeta = try { + schemaUtil.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema) + } catch { + case _: Exception => InternalSchema.getEmptyInternalSchema + } + (avroSchema, internalSchemaFromMeta) + } + + val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) + + val cdcExtractor: HoodieCDCExtractor = new HoodieCDCExtractor(metaClient, startInstant, endInstant) + + override final def needConversion: Boolean = false + + override def schema: StructType = CDCRelation.FULL_CDC_SPARK_SCHEMA + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + val internalRows = buildScan0(requiredColumns, filters) + internalRows.asInstanceOf[RDD[Row]] + } + + def buildScan0(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { + val nameToField = schema.fields.map(f => f.name -> f).toMap + val requiredSchema = StructType(requiredColumns.map(nameToField)) + val originTableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) + val parquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( + sparkSession = spark, + dataSchema = tableStructSchema, + partitionSchema = StructType(Nil), + requiredSchema = tableStructSchema, + filters = Nil, + options = options, + hadoopConf = spark.sessionState.newHadoopConf() + ) + + val changes = cdcExtractor.extractCDCFileSplits().values().asScala.map { pairs => + HoodieCDCFileGroupSplit( + pairs.asScala.map(pair => (pair.getLeft, pair.getRight)).sortBy(_._1).toArray + ) + } + val cdcRdd = new HoodieCDCRDD( + spark, + metaClient, + parquetReader, + originTableSchema, + schema, + requiredSchema, + changes.toArray + ) + cdcRdd.asInstanceOf[RDD[InternalRow]] + } +} + +object CDCRelation { + + val CDC_OPERATION_DELETE: UTF8String = UTF8String.fromString(DELETE.getValue) + val CDC_OPERATION_INSERT: UTF8String = UTF8String.fromString(INSERT.getValue) + val CDC_OPERATION_UPDATE: UTF8String = UTF8String.fromString(UPDATE.getValue) + + /** + * CDC Schema For Spark. + * Also it's schema when `hoodie.table.cdc.supplemental.logging.mode` is `cdc_data_before_after`. + * Here we use the debezium format. + */ + val FULL_CDC_SPARK_SCHEMA: StructType = { + StructType( + Seq( + StructField(CDC_OPERATION_TYPE, StringType), + StructField(CDC_COMMIT_TIMESTAMP, StringType), + StructField(CDC_BEFORE_IMAGE, StringType), + StructField(CDC_AFTER_IMAGE, StringType) + ) + ) + } + + /** + * CDC Schema For Spark when `hoodie.table.cdc.supplemental.logging.mode` is `op_key`. + */ + val MIN_CDC_SPARK_SCHEMA: StructType = { + StructType( + Seq( + StructField(CDC_OPERATION_TYPE, StringType), + StructField(CDC_RECORD_KEY, StringType) + ) + ) + } + + /** + * CDC Schema For Spark when `hoodie.table.cdc.supplemental.logging.mode` is `cdc_data_before`. + */ + val CDC_WITH_BEFORE_SPARK_SCHEMA: StructType = { + StructType( + Seq( + StructField(CDC_OPERATION_TYPE, StringType), + StructField(CDC_RECORD_KEY, StringType), + StructField(CDC_BEFORE_IMAGE, StringType) + ) + ) + } + + def isCDCEnabled(metaClient: HoodieTableMetaClient): Boolean = { + metaClient.getTableConfig.isCDCEnabled + } + + /** + * The only approach to create the CDC relation. + */ + def getCDCRelation( + sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, + options: Map[String, String]): CDCRelation = { + + if (!isCDCEnabled(metaClient)) { + throw new IllegalArgumentException(s"It isn't a CDC hudi table on ${metaClient.getBasePathV2.toString}") + } + + val startingInstant = options.getOrElse(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), + throw new HoodieException("CDC Query should provide the valid start version or timestamp") + ) + val endingInstant = options.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), + getTimestampOfLatestInstant(metaClient) + ) + if (startingInstant > endingInstant) { + throw new HoodieException(s"This is not a valid range between $startingInstant and $endingInstant") + } + + new CDCRelation(sqlContext, metaClient, startingInstant, endingInstant, options) + } + + def getTimestampOfLatestInstant(metaClient: HoodieTableMetaClient): String = { + val latestInstant = metaClient.getActiveTimeline.lastInstant() + if (latestInstant.isPresent) { + latestInstant.get().getTimestamp + } else { + throw new HoodieException("No valid instant in Active Timeline.") + } + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala new file mode 100644 index 0000000000000..49d5e19b1662d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala @@ -0,0 +1,618 @@ +/* + * 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.hudi.cdc + +import org.apache.hudi.HoodieBaseRelation.BaseFileReader +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, HoodieUnsafeRDD, LogFileIterator, RecordMergingFileIterator, SparkAdapterSupport} +import org.apache.hudi.HoodieConversionUtils._ +import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} +import org.apache.hudi.common.model.{FileSlice, HoodieLogFile, HoodieRecord, HoodieRecordPayload} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.cdc.HoodieCDCInferCase._ +import org.apache.hudi.common.table.cdc.HoodieCDCOperation._ +import org.apache.hudi.common.table.cdc.{HoodieCDCFileSplit, HoodieCDCSupplementalLoggingMode, HoodieCDCUtils} +import org.apache.hudi.common.table.log.HoodieCDCLogRecordIterator +import org.apache.hudi.common.table.timeline.HoodieInstant +import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.config.{HoodiePayloadConfig, HoodieWriteConfig} +import org.apache.hudi.keygen.constant.KeyGeneratorOptions +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory + +import com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Partition, SerializableWritable, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, SparkSession} +import org.apache.spark.sql.avro.HoodieAvroDeserializer +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.types.StringType +import org.apache.spark.unsafe.types.UTF8String + +import java.io.Closeable +import java.util.Properties +import java.util.stream.Collectors + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * The split that will be processed by spark task. + */ +case class HoodieCDCFileGroupSplit( + commitToChanges: Array[(HoodieInstant, HoodieCDCFileSplit)] +) + +/** + * The Spark [[Partition]]'s implementation. + */ +case class HoodieCDCFileGroupPartition( + index: Int, + split: HoodieCDCFileGroupSplit +) extends Partition + +class HoodieCDCRDD( + spark: SparkSession, + metaClient: HoodieTableMetaClient, + parquetReader: PartitionedFile => Iterator[InternalRow], + originTableSchema: HoodieTableSchema, + cdcSchema: StructType, + requiredCdcSchema: StructType, + changes: Array[HoodieCDCFileGroupSplit]) + extends RDD[InternalRow](spark.sparkContext, Nil) with HoodieUnsafeRDD { + + @transient private val hadoopConf = spark.sparkContext.hadoopConfiguration + + private val confBroadcast = spark.sparkContext.broadcast(new SerializableWritable(hadoopConf)) + + private val cdcSupplementalLoggingMode = HoodieCDCSupplementalLoggingMode.parse( + metaClient.getTableConfig.cdcSupplementalLoggingMode + ) + + private val props = HoodieFileIndex.getConfigProperties(spark, Map.empty) + + protected val payloadProps: Properties = Option(metaClient.getTableConfig.getPreCombineField) + .map { preCombineField => + HoodiePayloadConfig.newBuilder + .withPayloadOrderingField(preCombineField) + .build + .getProps + }.getOrElse(new Properties()) + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val cdcPartition = split.asInstanceOf[HoodieCDCFileGroupPartition] + new CDCFileGroupIterator(cdcPartition.split, metaClient) + } + + override protected def getPartitions: Array[Partition] = { + changes.zipWithIndex.map{ case (split, index) => + HoodieCDCFileGroupPartition(index, split) + }.toArray + } + + private class CDCFileGroupIterator( + split: HoodieCDCFileGroupSplit, + metaClient: HoodieTableMetaClient + ) extends Iterator[InternalRow] with SparkAdapterSupport with AvroDeserializerSupport with Closeable { + + private lazy val fs = metaClient.getFs.getFileSystem + + private lazy val conf = new Configuration(confBroadcast.value.value) + + private lazy val basePath = metaClient.getBasePathV2 + + private lazy val tableConfig = metaClient.getTableConfig + + private lazy val populateMetaFields = tableConfig.populateMetaFields() + + private lazy val keyGenerator = { + val props = new TypedProperties() + props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, tableConfig.getKeyGeneratorClassName) + props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key, tableConfig.getRecordKeyFieldProp) + props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key, tableConfig.getPartitionFieldProp) + HoodieSparkKeyGeneratorFactory.createKeyGenerator(props) + } + + private lazy val recordKeyField: String = if (populateMetaFields) { + HoodieRecord.RECORD_KEY_METADATA_FIELD + } else { + val keyFields = metaClient.getTableConfig.getRecordKeyFields.get() + checkState(keyFields.length == 1) + keyFields.head + } + + private lazy val preCombineFieldOpt: Option[String] = Option(metaClient.getTableConfig.getPreCombineField) + + private lazy val tableState = { + val metadataConfig = HoodieMetadataConfig.newBuilder() + .fromProperties(props) + .build(); + HoodieTableState( + pathToString(basePath), + split.commitToChanges.map(_._1.getTimestamp).max, + recordKeyField, + preCombineFieldOpt, + usesVirtualKeys = false, + metaClient.getTableConfig.getPayloadClass, + metadataConfig + ) + } + + private lazy val mapper: ObjectMapper = { + val _mapper = new ObjectMapper + _mapper.setSerializationInclusion(Include.NON_ABSENT) + _mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + _mapper.registerModule(DefaultScalaModule) + _mapper + } + + protected override val avroSchema: Schema = new Schema.Parser().parse(originTableSchema.avroSchemaStr) + + protected override val structTypeSchema: StructType = originTableSchema.structTypeSchema + + private lazy val serializer = sparkAdapter.createAvroSerializer(originTableSchema.structTypeSchema, + avroSchema, nullable = false) + + private lazy val reusableRecordBuilder: GenericRecordBuilder = new GenericRecordBuilder(avroSchema) + + private lazy val cdcAvroSchema: Schema = HoodieCDCUtils.schemaBySupplementalLoggingMode( + cdcSupplementalLoggingMode, + HoodieAvroUtils.removeMetadataFields(avroSchema) + ) + + private lazy val cdcSparkSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(cdcAvroSchema) + + /** + * The deserializer used to convert the CDC GenericRecord to Spark InternalRow. + */ + private lazy val cdcRecordDeserializer: HoodieAvroDeserializer = { + sparkAdapter.createAvroDeserializer(cdcAvroSchema, cdcSparkSchema) + } + + private lazy val projection: UnsafeProjection = generateUnsafeProjection(cdcSchema, requiredCdcSchema) + + // Iterator on cdc file + private val cdcFileIter = split.commitToChanges.sortBy(_._1).iterator + + // The instant that is currently being processed + private var currentInstant: HoodieInstant = _ + + // The change file that is currently being processed + private var currentChangeFile: HoodieCDCFileSplit = _ + + /** + * Two cases will use this to iterator the records: + * 1) extract the change data from the base file directly, including 'ADD_BASE_File' and 'REMOVE_BASE_File'. + * 2) when the type of cdc file is 'REPLACED_FILE_GROUP', + * use this to trace the records that are converted from the '[[beforeImageRecords]] + */ + private var recordIter: Iterator[InternalRow] = Iterator.empty + + /** + * Only one case where it will be used is that extract the change data from log files for mor table. + * At the time, 'logRecordIter' will work with [[beforeImageRecords]] that keep all the records of the previous file slice. + */ + private var logRecordIter: Iterator[(String, HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]])] = Iterator.empty + + /** + * Only one case where it will be used is that extract the change data from cdc log files. + */ + private var cdcLogRecordIterator: HoodieCDCLogRecordIterator = _ + + /** + * The next record need to be returned when call next(). + */ + protected var recordToLoad: InternalRow = _ + + /** + * The list of files to which 'beforeImageRecords' belong. + * Use it to determine if 'beforeImageRecords' contains all the required data that extract + * the change data from the current cdc file. + */ + private val beforeImageFiles: mutable.ArrayBuffer[String] = mutable.ArrayBuffer.empty + + /** + * Keep the before-image data. There cases will use this: + * 1) the cdc infer case is [[LOG_FILE]]; + * 2) the cdc infer case is [[AS_IS]] and [[cdcSupplementalLoggingMode]] is 'op_key'. + */ + private var beforeImageRecords: mutable.Map[String, GenericRecord] = mutable.Map.empty + + /** + * Keep the after-image data. Only one case will use this: + * the cdc infer case is [[AS_IS]] and [[cdcSupplementalLoggingMode]] is 'op_key' or 'cdc_data_before'. + */ + private var afterImageRecords: mutable.Map[String, InternalRow] = mutable.Map.empty + + private def needLoadNextFile: Boolean = { + !recordIter.hasNext && + !logRecordIter.hasNext && + (cdcLogRecordIterator == null || !cdcLogRecordIterator.hasNext) + } + + @tailrec final def hasNextInternal: Boolean = { + if (needLoadNextFile) { + loadCdcFile() + } + if (currentChangeFile == null) { + false + } else { + currentChangeFile.getCdcInferCase match { + case BASE_FILE_INSERT | BASE_FILE_DELETE | REPLACE_COMMIT => + if (recordIter.hasNext && loadNext()) { + true + } else { + hasNextInternal + } + case LOG_FILE => + if (logRecordIter.hasNext && loadNext()) { + true + } else { + hasNextInternal + } + case AS_IS => + if (cdcLogRecordIterator.hasNext && loadNext()) { + true + } else { + hasNextInternal + } + } + } + } + + override def hasNext: Boolean = hasNextInternal + + override final def next(): InternalRow = { + projection(recordToLoad) + } + + def loadNext(): Boolean = { + var loaded = false + currentChangeFile.getCdcInferCase match { + case BASE_FILE_INSERT => + val originRecord = recordIter.next() + recordToLoad.update(3, convertRowToJsonString(originRecord)) + loaded = true + case BASE_FILE_DELETE => + val originRecord = recordIter.next() + recordToLoad.update(2, convertRowToJsonString(originRecord)) + loaded = true + case LOG_FILE => + loaded = loadNextLogRecord() + case AS_IS => + val record = cdcLogRecordIterator.next().asInstanceOf[GenericRecord] + cdcSupplementalLoggingMode match { + case HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER => + recordToLoad.update(0, convertToUTF8String(String.valueOf(record.get(0)))) + val before = record.get(2).asInstanceOf[GenericRecord] + recordToLoad.update(2, convertToUTF8String(HoodieCDCUtils.recordToJson(before))) + val after = record.get(3).asInstanceOf[GenericRecord] + recordToLoad.update(3, convertToUTF8String(HoodieCDCUtils.recordToJson(after))) + case HoodieCDCSupplementalLoggingMode.WITH_BEFORE => + val row = cdcRecordDeserializer.deserialize(record).get.asInstanceOf[InternalRow] + val op = row.getString(0) + val recordKey = row.getString(1) + recordToLoad.update(0, convertToUTF8String(op)) + val before = record.get(2).asInstanceOf[GenericRecord] + recordToLoad.update(2, convertToUTF8String(HoodieCDCUtils.recordToJson(before))) + parse(op) match { + case INSERT => + recordToLoad.update(3, convertRowToJsonString(afterImageRecords(recordKey))) + case UPDATE => + recordToLoad.update(3, convertRowToJsonString(afterImageRecords(recordKey))) + case _ => + recordToLoad.update(3, null) + } + case _ => + val row = cdcRecordDeserializer.deserialize(record).get.asInstanceOf[InternalRow] + val op = row.getString(0) + val recordKey = row.getString(1) + recordToLoad.update(0, convertToUTF8String(op)) + parse(op) match { + case INSERT => + recordToLoad.update(2, null) + recordToLoad.update(3, convertRowToJsonString(afterImageRecords(recordKey))) + case UPDATE => + recordToLoad.update(2, convertRowToJsonString(deserialize(beforeImageRecords(recordKey)))) + recordToLoad.update(3, convertRowToJsonString(afterImageRecords(recordKey))) + case _ => + recordToLoad.update(2, convertRowToJsonString(deserialize(beforeImageRecords(recordKey)))) + recordToLoad.update(3, null) + } + } + loaded = true + case REPLACE_COMMIT => + val originRecord = recordIter.next() + recordToLoad.update(2, convertRowToJsonString(originRecord)) + loaded = true + } + loaded + } + + /** + * Load the next log record, and judege how to convert it to cdc format. + */ + private def loadNextLogRecord(): Boolean = { + var loaded = false + val (key, logRecord) = logRecordIter.next() + val indexedRecord = getInsertValue(logRecord) + if (indexedRecord.isEmpty) { + // it's a deleted record. + val existingRecordOpt = beforeImageRecords.remove(key) + if (existingRecordOpt.isEmpty) { + // no real record is deleted, just ignore. + logWarning("can not get any record that have the same key with the deleting logRecord.") + } else { + // there is a real record deleted. + recordToLoad.update(0, CDCRelation.CDC_OPERATION_DELETE) + recordToLoad.update(2, convertRowToJsonString(deserialize(existingRecordOpt.get))) + recordToLoad.update(3, null) + loaded = true + } + } else { + val existingRecordOpt = beforeImageRecords.get(key) + if (existingRecordOpt.isEmpty) { + // a new record is inserted. + val insertedRecord = convertIndexedRecordToRow(indexedRecord.get) + recordToLoad.update(0, CDCRelation.CDC_OPERATION_INSERT) + recordToLoad.update(2, null) + recordToLoad.update(3, convertRowToJsonString(insertedRecord)) + // insert into beforeImageRecords + beforeImageRecords(key) = serialize(insertedRecord) + loaded = true + } else { + // a existed record is updated. + val existingRecord = existingRecordOpt.get + val merged = merge(existingRecord, logRecord) + val mergeRow = convertIndexedRecordToRow(merged) + val existingRow = deserialize(existingRecord) + if (mergeRow != existingRow) { + recordToLoad.update(0, CDCRelation.CDC_OPERATION_UPDATE) + recordToLoad.update(2, convertRowToJsonString(existingRow)) + recordToLoad.update(3, convertRowToJsonString(mergeRow)) + // update into beforeImageRecords + beforeImageRecords(key) = serialize(mergeRow) + loaded = true + } + } + } + loaded + } + + private def loadCdcFile(): Unit = { + // reset all the iterator or reader first. + recordIter = Iterator.empty + logRecordIter = Iterator.empty + beforeImageRecords.clear() + afterImageRecords.clear() + if (cdcLogRecordIterator != null) { + cdcLogRecordIterator.close() + cdcLogRecordIterator = null + } + + if (cdcFileIter.hasNext) { + val pair = cdcFileIter.next() + currentInstant = pair._1 + currentChangeFile = pair._2 + currentChangeFile.getCdcInferCase match { + case BASE_FILE_INSERT => + assert(currentChangeFile.getCdcFile != null) + val absCDCPath = new Path(basePath, currentChangeFile.getCdcFile) + val fileStatus = fs.getFileStatus(absCDCPath) + val pf = PartitionedFile(InternalRow.empty, absCDCPath.toUri.toString, 0, fileStatus.getLen) + recordIter = parquetReader(pf) + case BASE_FILE_DELETE => + assert(currentChangeFile.getBeforeFileSlice.isPresent) + recordIter = loadFileSlice(currentChangeFile.getBeforeFileSlice.get) + case LOG_FILE => + assert(currentChangeFile.getCdcFile != null && currentChangeFile.getBeforeFileSlice.isPresent) + loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get) + val absLogPath = new Path(basePath, currentChangeFile.getCdcFile) + val morSplit = HoodieMergeOnReadFileSplit(None, List(new HoodieLogFile(fs.getFileStatus(absLogPath)))) + val logFileIterator = new LogFileIterator(morSplit, originTableSchema, originTableSchema, tableState, conf) + logRecordIter = logFileIterator.logRecordsIterator() + case AS_IS => + assert(currentChangeFile.getCdcFile != null) + // load beforeFileSlice to beforeImageRecords + if (currentChangeFile.getBeforeFileSlice.isPresent) { + loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get) + } + // load afterFileSlice to afterImageRecords + if (currentChangeFile.getAfterFileSlice.isPresent) { + val iter = loadFileSlice(currentChangeFile.getAfterFileSlice.get()) + afterImageRecords = mutable.Map.empty + iter.foreach { row => + val key = getRecordKey(row) + afterImageRecords.put(key, row.copy()) + } + } + val absCDCPath = new Path(basePath, currentChangeFile.getCdcFile) + cdcLogRecordIterator = new HoodieCDCLogRecordIterator(fs, absCDCPath, cdcAvroSchema) + case REPLACE_COMMIT => + if (currentChangeFile.getBeforeFileSlice.isPresent) { + loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get) + } + recordIter = beforeImageRecords.values.map { record => + deserialize(record) + }.iterator + beforeImageRecords.clear() + } + resetRecordFormat() + } else { + currentInstant = null + currentChangeFile = null + } + } + + /** + * Initialize the partial fields of the data to be returned in advance to speed up. + */ + private def resetRecordFormat(): Unit = { + recordToLoad = currentChangeFile.getCdcInferCase match { + case BASE_FILE_INSERT => + InternalRow.fromSeq(Array( + CDCRelation.CDC_OPERATION_INSERT, convertToUTF8String(currentInstant.getTimestamp), + null, null)) + case BASE_FILE_DELETE => + InternalRow.fromSeq(Array( + CDCRelation.CDC_OPERATION_DELETE, convertToUTF8String(currentInstant.getTimestamp), + null, null)) + case LOG_FILE => + InternalRow.fromSeq(Array( + null, convertToUTF8String(currentInstant.getTimestamp), + null, null)) + case AS_IS => + InternalRow.fromSeq(Array( + null, convertToUTF8String(currentInstant.getTimestamp), + null, null)) + case REPLACE_COMMIT => + InternalRow.fromSeq(Array( + CDCRelation.CDC_OPERATION_DELETE, convertToUTF8String(currentInstant.getTimestamp), + null, null)) + } + } + + /** + * If [[beforeImageFiles]] are the list of file that we want to load exactly, use this directly. + * Otherwise we need to re-load what we need. + */ + private def loadBeforeFileSliceIfNeeded(fileSlice: FileSlice): Unit = { + val files = List(fileSlice.getBaseFile.get().getPath) ++ + fileSlice.getLogFiles.collect(Collectors.toList[HoodieLogFile]).asScala + .map(f => pathToString(f.getPath)).toList + val same = files.sorted == beforeImageFiles.sorted.toList + if (!same) { + // clear up the beforeImageRecords + beforeImageRecords.clear() + val iter = loadFileSlice(fileSlice) + iter.foreach { row => + val key = getRecordKey(row) + beforeImageRecords.put(key, serialize(row)) + } + // reset beforeImageFiles + beforeImageFiles.clear() + beforeImageFiles.append(files: _*) + } + } + + private def loadFileSlice(fileSlice: FileSlice): Iterator[InternalRow] = { + val baseFileStatus = fs.getFileStatus(new Path(fileSlice.getBaseFile.get().getPath)) + val basePartitionedFile = PartitionedFile( + InternalRow.empty, + pathToString(baseFileStatus.getPath), + 0, + baseFileStatus.getLen + ) + val logFiles = fileSlice.getLogFiles + .sorted(HoodieLogFile.getLogFileComparator) + .collect(Collectors.toList[HoodieLogFile]) + .asScala.toList + .filterNot(_.getFileName.endsWith(HoodieCDCUtils.CDC_LOGFILE_SUFFIX)) + + if (logFiles.isEmpty) { + // no log files, just load the base parquet file + parquetReader(basePartitionedFile) + } else { + // use [[RecordMergingFileIterator]] to load both the base file and log files + val morSplit = HoodieMergeOnReadFileSplit(Some(basePartitionedFile), logFiles) + new RecordMergingFileIterator( + morSplit, + BaseFileReader(parquetReader, originTableSchema.structTypeSchema), + originTableSchema, + originTableSchema, + tableState, + conf) + } + } + + /** + * Convert InternalRow to json string. + */ + private def convertRowToJsonString(record: InternalRow): UTF8String = { + val map = scala.collection.mutable.Map.empty[String, Any] + originTableSchema.structTypeSchema.zipWithIndex.foreach { + case (field, idx) => + if (field.dataType.isInstanceOf[StringType]) { + map(field.name) = record.getString(idx) + } else { + map(field.name) = record.get(idx, field.dataType) + } + } + convertToUTF8String(mapper.writeValueAsString(map)) + } + + /** + * The data of string type is stored in InternalRow using UTF8String type. + */ + private def convertToUTF8String(str: String): UTF8String = { + UTF8String.fromString(str) + } + + private def pathToString(p: Path): String = { + p.toUri.toString + } + + private def serialize(curRowRecord: InternalRow): GenericRecord = { + serializer.serialize(curRowRecord).asInstanceOf[GenericRecord] + } + + private def getRecordKey(row: InternalRow): String = { + if (populateMetaFields) { + row.getString(structTypeSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)) + } else { + this.keyGenerator.getKey(serialize(row)).getRecordKey + } + } + + private def getInsertValue( + record: HoodieRecord[_ <: HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]]) + : Option[IndexedRecord] = { + toScalaOption(record.getData.getInsertValue(avroSchema, payloadProps)) + } + + private def convertIndexedRecordToRow(record: IndexedRecord): InternalRow = { + deserialize( + LogFileIterator.projectAvroUnsafe(record.asInstanceOf[GenericRecord], + avroSchema, reusableRecordBuilder) + ) + } + + private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): IndexedRecord = { + newRecord.getData.combineAndGetUpdateValue(curAvroRecord, avroSchema, payloadProps).get() + } + + private def generateUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = + HoodieCatalystExpressionUtils.generateUnsafeProjection(from, to) + + override def close(): Unit = {} + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala index bd8fdf263b3a1..49cbbdd799adf 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala @@ -23,12 +23,13 @@ import java.util.Date import org.apache.hadoop.fs.Path +import org.apache.hudi.cdc.CDCRelation import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport} import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.table.cdc.HoodieCDCUtils import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{FileIOUtils, TablePathUtils} - import org.apache.spark.sql.hudi.streaming.HoodieStreamSource.VERSION import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET import org.apache.spark.internal.Logging @@ -55,15 +56,24 @@ class HoodieStreamSource( extends Source with Logging with Serializable with SparkAdapterSupport { @transient private val hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + private lazy val tablePath: Path = { val path = new Path(parameters.getOrElse("path", "Missing 'path' option")) val fs = path.getFileSystem(hadoopConf) TablePathUtils.getTablePath(fs, path).get() } - private lazy val metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(tablePath.toString).build() + + private lazy val metaClient = HoodieTableMetaClient.builder() + .setConf(hadoopConf).setBasePath(tablePath.toString).build() + private lazy val tableType = metaClient.getTableType + private val isCDCQuery = CDCRelation.isCDCEnabled(metaClient) && + parameters.get(DataSourceReadOptions.QUERY_TYPE.key).contains(DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) && + parameters.get(DataSourceReadOptions.INCREMENTAL_FORMAT.key).contains(DataSourceReadOptions.INCREMENTAL_FORMAT_CDC_VAL) + @transient private var lastOffset: HoodieSourceOffset = _ + @transient private lazy val initialOffsets = { val metadataLog = new HDFSMetadataLog[HoodieSourceOffset](sqlContext.sparkSession, metadataPath) { @@ -117,9 +127,13 @@ class HoodieStreamSource( } override def schema: StructType = { - schemaOption.getOrElse { - val schemaUtil = new TableSchemaResolver(metaClient) - AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) + if (isCDCQuery) { + CDCRelation.FULL_CDC_SPARK_SCHEMA + } else { + schemaOption.getOrElse { + val schemaUtil = new TableSchemaResolver(metaClient) + AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema) + } } } @@ -152,27 +166,38 @@ class HoodieStreamSource( sqlContext.internalCreateDataFrame( sqlContext.sparkContext.emptyRDD[InternalRow].setName("empty"), schema, isStreaming = true) } else { - // Consume the data between (startCommitTime, endCommitTime] - val incParams = parameters ++ Map( - DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, - DataSourceReadOptions.BEGIN_INSTANTTIME.key -> startCommitTime(startOffset), - DataSourceReadOptions.END_INSTANTTIME.key -> endOffset.commitTime - ) - - val rdd = tableType match { - case HoodieTableType.COPY_ON_WRITE => - val serDe = sparkAdapter.createSparkRowSerDe(schema) - new IncrementalRelation(sqlContext, incParams, Some(schema), metaClient) - .buildScan() - .map(serDe.serializeRow) - case HoodieTableType.MERGE_ON_READ => - val requiredColumns = schema.fields.map(_.name) - new MergeOnReadIncrementalRelation(sqlContext, incParams, Some(schema), metaClient) - .buildScan(requiredColumns, Array.empty[Filter]) - .asInstanceOf[RDD[InternalRow]] - case _ => throw new IllegalArgumentException(s"UnSupport tableType: $tableType") + if (isCDCQuery) { + val cdcOptions = Map( + DataSourceReadOptions.BEGIN_INSTANTTIME.key()-> startCommitTime(startOffset), + DataSourceReadOptions.END_INSTANTTIME.key() -> endOffset.commitTime + ) + val rdd = CDCRelation.getCDCRelation(sqlContext, metaClient, cdcOptions) + .buildScan0(HoodieCDCUtils.CDC_COLUMNS, Array.empty) + + sqlContext.sparkSession.internalCreateDataFrame(rdd, CDCRelation.FULL_CDC_SPARK_SCHEMA, isStreaming = true) + } else { + // Consume the data between (startCommitTime, endCommitTime] + val incParams = parameters ++ Map( + DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, + DataSourceReadOptions.BEGIN_INSTANTTIME.key -> startCommitTime(startOffset), + DataSourceReadOptions.END_INSTANTTIME.key -> endOffset.commitTime + ) + + val rdd = tableType match { + case HoodieTableType.COPY_ON_WRITE => + val serDe = sparkAdapter.createSparkRowSerDe(schema) + new IncrementalRelation(sqlContext, incParams, Some(schema), metaClient) + .buildScan() + .map(serDe.serializeRow) + case HoodieTableType.MERGE_ON_READ => + val requiredColumns = schema.fields.map(_.name) + new MergeOnReadIncrementalRelation(sqlContext, incParams, Some(schema), metaClient) + .buildScan(requiredColumns, Array.empty[Filter]) + .asInstanceOf[RDD[InternalRow]] + case _ => throw new IllegalArgumentException(s"UnSupport tableType: $tableType") + } + sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true) } - sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala new file mode 100644 index 0000000000000..5f447a9bce6cc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala @@ -0,0 +1,206 @@ +/* + * 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.hudi.functional.cdc + +import org.apache.hudi.DataSourceReadOptions._ +import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieKey, HoodieLogFile, HoodieRecord} +import org.apache.hudi.common.table.cdc.HoodieCDCOperation +import org.apache.hudi.common.table.HoodieTableConfig +import org.apache.hudi.common.table.log.HoodieLogFormat +import org.apache.hudi.common.table.log.block.HoodieDataBlock +import org.apache.hudi.common.table.timeline.HoodieInstant +import org.apache.hudi.common.testutils.RawTripTestPayload +import org.apache.hudi.config.{HoodieCleanConfig, HoodieWriteConfig} +import org.apache.hudi.testutils.HoodieClientTestBase + +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericRecord, IndexedRecord} + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{DataFrame, SparkSession} + +import org.junit.jupiter.api.{AfterEach, BeforeEach} +import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNull, assertTrue} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +abstract class HoodieCDCTestBase extends HoodieClientTestBase { + + var spark: SparkSession = _ + + val commonOpts = Map( + HoodieTableConfig.CDC_ENABLED.key -> "true", + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + "hoodie.bulkinsert.shuffle.parallelism" -> "2", + "hoodie.delete.shuffle.parallelism" -> "1", + RECORDKEY_FIELD.key -> "_row_key", + PRECOMBINE_FIELD.key -> "timestamp", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1", + HoodieCleanConfig.AUTO_CLEAN.key -> "false" + ) + + @BeforeEach override def setUp(): Unit = { + setTableName("hoodie_test") + initPath() + initSparkContexts() + spark = sqlContext.sparkSession + initTestDataGenerator() + initFileSystem() + } + + @AfterEach override def tearDown(): Unit = { + cleanupSparkContexts() + cleanupTestDataGenerator() + cleanupFileSystem() + } + + protected def cdcDataFrame(basePath: String, startingInstant: String, endingInstant: String): DataFrame = { + val reader = spark.read.format("hudi") + .option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(INCREMENTAL_FORMAT.key, INCREMENTAL_FORMAT_CDC_VAL) + .option("hoodie.datasource.read.begin.instanttime", startingInstant) + if (endingInstant != null) { + reader.option("hoodie.datasource.read.end.instanttime", endingInstant) + } + reader.load(basePath) + } + + protected def cdcDataFrame(startingInstant: String, endingInstant: String = null): DataFrame = { + cdcDataFrame(basePath, startingInstant, endingInstant) + } + + /** + * whether this instant will create a cdc log file. + */ + protected def hasCDCLogFile(instant: HoodieInstant): Boolean = { + val commitMetadata = HoodieCommitMetadata.fromBytes( + metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + classOf[HoodieCommitMetadata] + ) + val hoodieWriteStats = commitMetadata.getWriteStats.asScala + hoodieWriteStats.exists { hoodieWriteStat => + val cdcPath = hoodieWriteStat.getCdcPath + cdcPath != null && cdcPath.nonEmpty + } + } + + /** + * whether this instant will create a cdc log file. + */ + protected def getCDCLogFIle(instant: HoodieInstant): List[String] = { + val commitMetadata = HoodieCommitMetadata.fromBytes( + metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + classOf[HoodieCommitMetadata] + ) + commitMetadata.getWriteStats.asScala.map(_.getCdcPath).toList + } + + protected def readCDCLogFile(relativeLogFile: String, cdcSchema: Schema): List[IndexedRecord] = { + val logFile = new HoodieLogFile( + metaClient.getFs.getFileStatus(new Path(metaClient.getBasePathV2, relativeLogFile))) + val reader = HoodieLogFormat.newReader(fs, logFile, cdcSchema); + assertTrue(reader.hasNext); + + val block = reader.next().asInstanceOf[HoodieDataBlock]; + block.getRecordIterator.asScala.toList + } + + protected def checkCDCDataForInsertOrUpdate(cdcSupplementalLoggingMode: String, + cdcSchema: Schema, + dataSchema: Schema, + cdcRecords: Seq[IndexedRecord], + newHoodieRecords: java.util.List[HoodieRecord[_]], + op: HoodieCDCOperation): Unit = { + val cdcRecord = cdcRecords.head.asInstanceOf[GenericRecord] + // check schema + assertEquals(cdcRecord.getSchema, cdcSchema) + if (cdcSupplementalLoggingMode == "cdc_op_key") { + // check record key + assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) + } else if (cdcSupplementalLoggingMode == "cdc_data_before") { + // check record key + assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) + // check before + if (op == HoodieCDCOperation.INSERT) { + assertNull(cdcRecord.get("before")) + } else { + val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcRecord.get("record_key").toString).get + .getData.asInstanceOf[RawTripTestPayload] + val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] + val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] + assertNotEquals(genericRecord.get("begin_lat"), cdcBeforeValue.get("begin_lat")) + } + } else { + val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] + val cdcAfterValue = cdcRecord.get("after").asInstanceOf[GenericRecord] + if (op == HoodieCDCOperation.INSERT) { + // check before + assertNull(cdcBeforeValue) + // check after + val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcAfterValue.get("_row_key").toString).get + .getData.asInstanceOf[RawTripTestPayload] + val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] + assertEquals(genericRecord.get("begin_lat"), cdcAfterValue.get("begin_lat")) + } else { + val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcAfterValue.get("_row_key").toString).get + .getData.asInstanceOf[RawTripTestPayload] + val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] + // check before + assertNotEquals(genericRecord.get("begin_lat"), cdcBeforeValue.get("begin_lat")) + // check after + assertEquals(genericRecord.get("begin_lat"), cdcAfterValue.get("begin_lat")) + } + } + } + + protected def checkCDCDataForDelete(cdcSupplementalLoggingMode: String, + cdcSchema: Schema, + cdcRecords: Seq[IndexedRecord], + deletedKeys: java.util.List[HoodieKey]): Unit = { + val cdcRecord = cdcRecords.head.asInstanceOf[GenericRecord] + // check schema + assertEquals(cdcRecord.getSchema, cdcSchema) + if (cdcSupplementalLoggingMode == "cdc_op_key") { + // check record key + assert(cdcRecords.map(_.get(1).toString).sorted == deletedKeys.map(_.getRecordKey).sorted) + } else if (cdcSupplementalLoggingMode == "cdc_data_before") { + // check record key + assert(cdcRecords.map(_.get(1).toString).sorted == deletedKeys.map(_.getRecordKey).sorted) + } else { + val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] + val cdcAfterValue = cdcRecord.get("after").asInstanceOf[GenericRecord] + // check before + assert(deletedKeys.exists(_.getRecordKey == cdcBeforeValue.get("_row_key").toString)) + // check after + assertNull(cdcAfterValue) + } + } + + protected def assertCDCOpCnt(cdcData: DataFrame, expectedInsertCnt: Long, + expectedUpdateCnt: Long, expectedDeletedCnt: Long): Unit = { + assertEquals(expectedInsertCnt, cdcData.where("op = 'i'").count()) + assertEquals(expectedUpdateCnt, cdcData.where("op = 'u'").count()) + assertEquals(expectedDeletedCnt, cdcData.where("op = 'd'").count()) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala index 6f0731578d634..40d156d6902fc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala @@ -18,67 +18,30 @@ package org.apache.hudi.functional.cdc -import org.apache.avro.Schema -import org.apache.avro.generic.{GenericRecord, IndexedRecord} - -import org.apache.hadoop.fs.Path - -import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.config.HoodieMetadataConfig -import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieKey, HoodieLogFile, HoodieRecord} +import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.common.table.cdc.{HoodieCDCOperation, HoodieCDCSupplementalLoggingMode, HoodieCDCUtils} -import org.apache.hudi.common.table.log.HoodieLogFormat -import org.apache.hudi.common.table.log.block.{HoodieDataBlock} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.HoodieInstant -import org.apache.hudi.common.testutils.RawTripTestPayload +import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrings, recordsToStrings} -import org.apache.hudi.config.{HoodieCleanConfig, HoodieWriteConfig} -import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.SaveMode -import org.junit.jupiter.api.{AfterEach, BeforeEach} -import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNull, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.CsvSource import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ - -class TestCDCDataFrameSuite extends HoodieClientTestBase { - - var spark: SparkSession = _ - - val commonOpts = Map( - HoodieTableConfig.CDC_ENABLED.key -> "true", - "hoodie.insert.shuffle.parallelism" -> "4", - "hoodie.upsert.shuffle.parallelism" -> "4", - "hoodie.bulkinsert.shuffle.parallelism" -> "2", - "hoodie.delete.shuffle.parallelism" -> "1", - RECORDKEY_FIELD.key -> "_row_key", - PRECOMBINE_FIELD.key -> "timestamp", - HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", - HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1", - HoodieCleanConfig.AUTO_CLEAN.key -> "false" - ) - - @BeforeEach override def setUp(): Unit = { - setTableName("hoodie_test") - initPath() - initSparkContexts() - spark = sqlContext.sparkSession - initTestDataGenerator() - initFileSystem() - } - @AfterEach override def tearDown(): Unit = { - cleanupSparkContexts() - cleanupTestDataGenerator() - cleanupFileSystem() - } +class TestCDCDataFrameSuite extends HoodieCDCTestBase { + /** + * Step1: Insert 100 + * Step2: Upsert 50 + * Step3: Delete 20 With Clustering + * Step4: Insert Overwrite 50 + * Step5: Upsert 30 With Clean + * Step6: Bluk_Insert 20 + */ @ParameterizedTest @CsvSource(Array("cdc_op_key", "cdc_data_before", "cdc_data_before_after")) def testCOWDataSourceWrite(cdcSupplementalLoggingMode: String): Unit = { @@ -86,6 +49,11 @@ class TestCDCDataFrameSuite extends HoodieClientTestBase { HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key -> cdcSupplementalLoggingMode ) + var totalInsertedCnt = 0L + var totalUpdatedCnt = 0L + var totalDeletedCnt = 0L + var allVisibleCDCData = spark.emptyDataFrame + // Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -98,16 +66,20 @@ class TestCDCDataFrameSuite extends HoodieClientTestBase { .setBasePath(basePath) .setConf(spark.sessionState.newHadoopConf) .build() - val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() - assertEquals(spark.read.format("hudi").load(basePath).count(), 100) - // all the data is new-coming, it will write out cdc log files. - assertFalse(hasCDCLogFile(instant1)) val schemaResolver = new TableSchemaResolver(metaClient) val dataSchema = schemaResolver.getTableAvroSchema(false) val cdcSchema = HoodieCDCUtils.schemaBySupplementalLoggingMode( HoodieCDCSupplementalLoggingMode.parse(cdcSupplementalLoggingMode), dataSchema) + totalInsertedCnt += 100 + val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() + // all the data is new-coming, it will write out cdc log files. + assertFalse(hasCDCLogFile(instant1)) + val commitTime1 = instant1.getTimestamp + val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) + // Upsert Operation val hoodieRecords2 = dataGen.generateUniqueUpdates("001", 50) val records2 = recordsToStrings(hoodieRecords2).toList @@ -117,50 +89,153 @@ class TestCDCDataFrameSuite extends HoodieClientTestBase { .mode(SaveMode.Append) .save(basePath) val instant2 = metaClient.reloadActiveTimeline.lastInstant().get() - assertEquals(spark.read.format("hudi").load(basePath).count(), 100) // part of data are updated, it will write out cdc log files assertTrue(hasCDCLogFile(instant2)) - val cdcData2: Seq[IndexedRecord] = getCDCLogFIle(instant2).flatMap(readCDCLogFile(_, cdcSchema)) + // check cdc data + val cdcDataFromCDCLogFile2 = getCDCLogFIle(instant2).flatMap(readCDCLogFile(_, cdcSchema)) // check the num of cdc data - assertEquals(cdcData2.size, 50) - // check op - assert(cdcData2.forall( r => r.get(0).toString == "u")) + assertEquals(cdcDataFromCDCLogFile2.size, 50) // check record key, before, after according to the supplemental logging mode checkCDCDataForInsertOrUpdate(cdcSupplementalLoggingMode, cdcSchema, dataSchema, - cdcData2, hoodieRecords2, HoodieCDCOperation.UPDATE) - - // Delete Operation - val hoodieKey3 = dataGen.generateUniqueDeletes(20) - val records3 = deleteRecordsToStrings(hoodieKey3).toList + cdcDataFromCDCLogFile2, hoodieRecords2, HoodieCDCOperation.UPDATE) + + val commitTime2 = instant2.getTimestamp + var currentSnapshotData = spark.read.format("hudi").load(basePath) + // at the last commit, 100 records are inserted. + val insertedCnt2 = currentSnapshotData.count() - 100 + val updatedCnt2 = 50 - insertedCnt2 + val cdcDataOnly2 = cdcDataFrame((commitTime2.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly2, insertedCnt2, updatedCnt2, 0) + totalUpdatedCnt += updatedCnt2 + totalInsertedCnt += insertedCnt2 + + // Delete Operation With Clustering Operation + val records3 = deleteRecordsToStrings(dataGen.generateUniqueDeletes(20)).toList val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") .options(options) - .option(OPERATION.key, DELETE_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL) + .option("hoodie.clustering.inline", "true") + .option("hoodie.clustering.inline.max.commits", "1") .mode(SaveMode.Append) .save(basePath) val instant3 = metaClient.reloadActiveTimeline.lastInstant().get() - assertEquals(spark.read.format("hudi").load(basePath).count(), 80) - - // part of data are deleted, it will write out cdc log files - assertTrue(hasCDCLogFile(instant3)) - val cdcData3 = getCDCLogFIle(instant3).flatMap(readCDCLogFile(_, cdcSchema)) - // check the num of cdc data - assertEquals(cdcData3.size, 20) - // check op - assert(cdcData3.forall( r => r.get(0).toString == "d")) - // check record key, before, after according to the supplemental logging mode - checkCDCDataForDelete(cdcSupplementalLoggingMode, cdcSchema, cdcData3, hoodieKey3) + // only part of data are deleted and some data will write back to the file. + // it will write out cdc log files. But instant3 is the clustering instant, not the delete one. so we omit to test. + val commitTime3 = instant3.getTimestamp + currentSnapshotData = spark.read.format("hudi").load(basePath) + // here we use `commitTime2` to query the change data in commit 3. + // because `commitTime3` is the ts of the clustering operation, not the delete operation. + val cdcDataOnly3 = cdcDataFrame(commitTime2) + assertCDCOpCnt(cdcDataOnly3, 0, 0, 20) + totalDeletedCnt += 20 + + // all the change data in the range [commitTime1, commitTime3] + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // check both starting and ending are provided + val cdcDataFrom2To3 = cdcDataFrame(commitTime1, commitTime3) + assertCDCOpCnt(cdcDataFrom2To3, insertedCnt2, updatedCnt2, 20) + + // Insert Overwrite Operation + val records4 = recordsToStrings(dataGen.generateInserts("003", 50)).toList + val inputDF4 = spark.read.json(spark.sparkContext.parallelize(records4, 2)) + inputDF4.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val instant4 = metaClient.reloadActiveTimeline.lastInstant().get() + // the files which keep all the old data will be replaced directly. + // and all the new data will write out some new file groups. + // it will NOT write out cdc log files + assertFalse(hasCDCLogFile(instant4)) + val commitTime4 = instant4.getTimestamp + val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) + val insertedCnt4 = 50 + val deletedCnt4 = currentSnapshotData.count() + assertCDCOpCnt(cdcDataOnly4, insertedCnt4, 0, deletedCnt4) + totalInsertedCnt += insertedCnt4 + totalDeletedCnt += deletedCnt4 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // Upsert Operation With Clean Operation + val records5 = recordsToStrings(dataGen.generateUniqueUpdates("004", 30)).toList + val inputDF5 = spark.read.json(spark.sparkContext.parallelize(records5, 2)) + inputDF5.write.format("org.apache.hudi") + .options(options) + .option("hoodie.clean.automatic", "true") + .option("hoodie.keep.min.commits", "2") + .option("hoodie.keep.max.commits", "3") + .option("hoodie.cleaner.commits.retained", "1") + .mode(SaveMode.Append) + .save(basePath) + val instant5 = metaClient.reloadActiveTimeline.lastInstant().get() + // part of data are updated, it will write out cdc log files. + // But instant5 is the clean instant, not the upsert one. so we omit to test. + val commitTime5 = instant5.getTimestamp + // here we use `commitTime4` to query the change data in commit 5. + // because `commitTime5` is the ts of the clean operation, not the upsert operation. + val cdcDataOnly5 = cdcDataFrame(commitTime4) + val currentData = spark.read.format("hudi").load(basePath) + val insertedCnt5 = currentData.count() - 50 + val updatedCnt5 = 30 - insertedCnt5 + assertCDCOpCnt(cdcDataOnly5, insertedCnt5, updatedCnt5, 0) + // here cause we do the clean operation and just remain the commit4 and commit5, so we need to reset the total cnt. + // 50 is the number of inserted records at commit 4. + totalInsertedCnt = 50 + insertedCnt5 + totalUpdatedCnt = updatedCnt5 + totalDeletedCnt = 0 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // Bulk_Insert Operation With Clean Operation + val records6 = recordsToStrings(dataGen.generateInserts("005", 20)).toList + val inputDF6 = spark.read.json(spark.sparkContext.parallelize(records6, 2)) + inputDF6.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val instant6 = metaClient.reloadActiveTimeline.lastInstant().get() + // the files which keep all the old data will be replaced directly. + // and all the new data will write out some new file groups. + // it will NOT write out cdc log files + assertFalse(hasCDCLogFile(instant6)) + val commitTime6 = instant6.getTimestamp + val cdcDataOnly6 = cdcDataFrame((commitTime6.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly6, 20, 0, 0) + totalInsertedCnt += 20 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) } + + /** + * Step1: Insert 100 + * Step2: Upsert 50 + * Step3: Delete 20 With Compaction + * Step4: Bluk_Insert 100 + * Step5: Upsert 60 With Clustering + * Step6: Insert Overwrite 70 + * Step7: Upsert 30 With CLean + */ @ParameterizedTest @CsvSource(Array("cdc_op_key", "cdc_data_before", "cdc_data_before_after")) def testMORDataSourceWrite(cdcSupplementalLoggingMode: String): Unit = { val options = commonOpts ++ Map( - TABLE_TYPE.key() -> MOR_TABLE_TYPE_OPT_VAL, + DataSourceWriteOptions.TABLE_TYPE.key() -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key -> cdcSupplementalLoggingMode ) + var totalInsertedCnt = 0L + var totalUpdatedCnt = 0L + var totalDeletedCnt = 0L + var allVisibleCDCData = spark.emptyDataFrame + // 1. Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) @@ -179,9 +254,13 @@ class TestCDCDataFrameSuite extends HoodieClientTestBase { val cdcSchema = HoodieCDCUtils.schemaBySupplementalLoggingMode( HoodieCDCSupplementalLoggingMode.parse(cdcSupplementalLoggingMode), dataSchema) + totalInsertedCnt += 100 val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() - // all the data is new-coming, it will NOT write out cdc log files. + // all the data is new-coming, it will write out cdc log files. assertFalse(hasCDCLogFile(instant1)) + val commitTime1 = instant1.getTimestamp + val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) // 2. Upsert Operation val records2_1 = recordsToStrings(dataGen.generateUniqueUpdates("001", 30)).toList @@ -196,129 +275,271 @@ class TestCDCDataFrameSuite extends HoodieClientTestBase { // part of data are updated, it will write out cdc log files assertTrue(hasCDCLogFile(instant2)) - val cdcData2 = getCDCLogFIle(instant2).flatMap(readCDCLogFile(_, cdcSchema)) - assertEquals(cdcData2.size, 50) + val cdcDataFromCDCLogFile2 = getCDCLogFIle(instant2).flatMap(readCDCLogFile(_, cdcSchema)) + assertEquals(cdcDataFromCDCLogFile2.size, 50) // check op - assertEquals(cdcData2.count(r => r.get(0).toString == "u"), 30) - assertEquals(cdcData2.count(r => r.get(0).toString == "i"), 20) - - // 3. Delete Operation + assertEquals(cdcDataFromCDCLogFile2.count(r => r.get(0).toString == "u"), 30) + assertEquals(cdcDataFromCDCLogFile2.count(r => r.get(0).toString == "i"), 20) + + val commitTime2 = instant2.getTimestamp + var currentSnapshotData = spark.read.format("hudi").load(basePath) + // at the last commit, 100 records are inserted. + val insertedCnt2 = currentSnapshotData.count() - 100 + val updatedCnt2 = 50 - insertedCnt2 + val cdcDataOnly2 = cdcDataFrame((commitTime2.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly2, insertedCnt2, updatedCnt2, 0) + totalUpdatedCnt += updatedCnt2 + totalInsertedCnt += insertedCnt2 + + // 3. Delete Operation With Compaction Operation val records3 = deleteRecordsToStrings(dataGen.generateUniqueDeletes(20)).toList val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) inputDF3.write.format("org.apache.hudi") .options(options) - .option(OPERATION.key, DELETE_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL) + .option("hoodie.compact.inline", "true") + .option("hoodie.compact.inline.max.delta.commits", "1") .mode(SaveMode.Append) .save(basePath) val instant3 = metaClient.reloadActiveTimeline.lastInstant().get() // in cases that there is log files, it will NOT write out cdc log files. - assertFalse(hasCDCLogFile(instant3)) + // But instant3 is the compaction instant, not the delete one. so we omit to test. + val commitTime3 = instant3.getTimestamp + currentSnapshotData = spark.read.format("hudi").load(basePath) + // here we use `commitTime2` to query the change data in commit 3. + // because `commitTime3` is the ts of the clustering operation, not the delete operation. + val cdcDataOnly3 = cdcDataFrame(commitTime2) + assertCDCOpCnt(cdcDataOnly3, 0, 0, 20) + + totalDeletedCnt += 20 + // all the change data in the range [commitTime1, commitTime3] + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // 4. Bulk_Insert Operation + val records4 = recordsToStrings(dataGen.generateInserts("003", 100)).toList + val inputDF4 = spark.read.json(spark.sparkContext.parallelize(records4, 2)) + inputDF4.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val instant4 = metaClient.reloadActiveTimeline.lastInstant().get() + // all the new data will write out some new file groups. + // it will NOT write out cdc log files + assertFalse(hasCDCLogFile(instant4)) + val commitTime4 = instant4.getTimestamp + val cntForInstant4 = spark.read.format("hudi").load(basePath).count() + val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) + val insertedCnt4 = 100 + assertCDCOpCnt(cdcDataOnly4, insertedCnt4, 0, 0) + + totalInsertedCnt += insertedCnt4 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // 5. Upsert Operation With Clustering Operation + val records5 = recordsToStrings(dataGen.generateUniqueUpdates("004", 60)).toList + val inputDF5 = spark.read.json(spark.sparkContext.parallelize(records5, 2)) + inputDF5.write.format("org.apache.hudi") + .options(options) + .option("hoodie.clustering.inline", "true") + .option("hoodie.clustering.inline.max.commits", "1") + .mode(SaveMode.Append) + .save(basePath) + val instant5 = metaClient.reloadActiveTimeline.lastInstant().get() + // in cases that there is log files, it will NOT write out cdc log files. + // But instant5 is the clustering instant, not the upsert one. so we omit to test. + val commitTime5 = instant5.getTimestamp + // here we use `commitTime4` to query the change data in commit 5. + // because `commitTime5` is the ts of the clean operation, not the upsert operation. + val cdcDataOnly5 = cdcDataFrame(commitTime4) + val cntForInstant5 = spark.read.format("hudi").load(basePath).count() + val insertedCnt5 = cntForInstant5 - cntForInstant4 + val updatedCnt5 = 60 - insertedCnt5 + assertCDCOpCnt(cdcDataOnly5, insertedCnt5, updatedCnt5, 0) + + totalInsertedCnt += insertedCnt5 + totalUpdatedCnt += updatedCnt5 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // check both starting and ending are provided + val cdcDataFrom3To4 = cdcDataFrame(commitTime2, commitTime4) + assertCDCOpCnt(cdcDataFrom3To4, insertedCnt4, 0, 20) + + // 6. Insert Overwrite Operation + val records6 = recordsToStrings(dataGen.generateInserts("005", 70)).toList + val inputDF6 = spark.read.json(spark.sparkContext.parallelize(records6, 2)) + inputDF6.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val instant6 = metaClient.reloadActiveTimeline.lastInstant().get() + // the files which keep all the old data will be replaced directly. + // and all the new data will write out some new file groups. + // it will NOT write out cdc log files + assertFalse(hasCDCLogFile(instant6)) + val commitTime6 = instant6.getTimestamp + val cntForInstant6 = spark.read.format("hudi").load(basePath).count() + val cdcDataOnly6 = cdcDataFrame((commitTime6.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly6, 70, 0, cntForInstant5) + totalInsertedCnt += 70 + totalDeletedCnt += cntForInstant5 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // 7. Upsert Operation With Clean Operation + val records7 = recordsToStrings(dataGen.generateUniqueUpdates("006", 30)).toList + val inputDF7 = spark.read.json(spark.sparkContext.parallelize(records7, 2)) + inputDF7.write.format("org.apache.hudi") + .options(options) + .option("hoodie.clean.automatic", "true") + .option("hoodie.keep.min.commits", "2") + .option("hoodie.keep.max.commits", "3") + .option("hoodie.cleaner.commits.retained", "1") + .mode(SaveMode.Append) + .save(basePath) + val instant7 = metaClient.reloadActiveTimeline.lastInstant().get() + // in cases that there is log files, it will NOT write out cdc log files. + // But instant7 is the clean instant, not the upsert one. so we omit to test. + val commitTime7 = instant7.getTimestamp + val cntForInstant7 = spark.read.format("hudi").load(basePath).count() + val cdcDataOnly7 = cdcDataFrame(commitTime6) + val insertedCnt7 = cntForInstant7 - cntForInstant6 + val updatedCnt7 = 30 - insertedCnt7 + assertCDCOpCnt(cdcDataOnly7, insertedCnt7, updatedCnt7, 0) + + // here cause we do the clean operation and just remain the commit6 and commit7, so we need to reset the total cnt. + // 70 is the number of inserted records at commit 6. + totalInsertedCnt = 70 + insertedCnt7 + totalUpdatedCnt = updatedCnt7 + totalDeletedCnt = 0 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) } /** - * whether this instant will create a cdc log file. + * Step1: Insert Data 100 + * Step2: Insert Overwrite Partition + * Step3: Delete Partition + * Step4: Upsert */ - private def hasCDCLogFile(instant: HoodieInstant): Boolean = { - val commitMetadata = HoodieCommitMetadata.fromBytes( - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), - classOf[HoodieCommitMetadata] + @ParameterizedTest + @CsvSource(Array( + "COPY_ON_WRITE,cdc_data_before_after", "MERGE_ON_READ,cdc_data_before_after", + "COPY_ON_WRITE,cdc_data_before", "MERGE_ON_READ,cdc_data_before", + "COPY_ON_WRITE,cdc_op_key", "MERGE_ON_READ,cdc_op_key")) + def testDataSourceWriteWithPartitionField(tableType: String, cdcSupplementalLoggingMode: String): Unit = { + val options = commonOpts ++ Map( + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", + DataSourceWriteOptions.TABLE_TYPE.key -> tableType, + HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key -> cdcSupplementalLoggingMode ) - val hoodieWriteStats = commitMetadata.getWriteStats.asScala - hoodieWriteStats.exists { hoodieWriteStat => - val cdcPath = hoodieWriteStat.getCdcPath - cdcPath != null && cdcPath.nonEmpty - } - } - /** - * whether this instant will create a cdc log file. - */ - private def getCDCLogFIle(instant: HoodieInstant): List[String] = { - val commitMetadata = HoodieCommitMetadata.fromBytes( - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), - classOf[HoodieCommitMetadata] - ) - commitMetadata.getWriteStats.asScala.map(_.getCdcPath).toList - } + var totalInsertedCnt = 0L + var totalUpdatedCnt = 0L + var totalDeletedCnt = 0L + var allVisibleCDCData = spark.emptyDataFrame - private def readCDCLogFile(relativeLogFile: String, cdcSchema: Schema): List[IndexedRecord] = { - val logFile = new HoodieLogFile( - metaClient.getFs.getFileStatus(new Path(metaClient.getBasePathV2, relativeLogFile))) - val reader = HoodieLogFormat.newReader(fs, logFile, cdcSchema); - assertTrue(reader.hasNext); + // Insert Operation + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(options) + .mode(SaveMode.Overwrite) + .save(basePath) + val partitionToCnt = spark.read.format("hudi").load(basePath) + .groupBy("partition").count().collect() + .map(row => row.getString(0) -> row.getLong(1)).toMap + assert(partitionToCnt.contains(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)) + assert(partitionToCnt.contains(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)) - val block = reader.next().asInstanceOf[HoodieDataBlock]; - block.getRecordIterator.asScala.toList - } + // init meta client + metaClient = HoodieTableMetaClient.builder() + .setBasePath(basePath) + .setConf(spark.sessionState.newHadoopConf) + .build() - private def checkCDCDataForInsertOrUpdate(cdcSupplementalLoggingMode: String, - cdcSchema: Schema, - dataSchema: Schema, - cdcRecords: Seq[IndexedRecord], - newHoodieRecords: java.util.List[HoodieRecord[_]], - op: HoodieCDCOperation): Unit = { - val cdcRecord = cdcRecords.head.asInstanceOf[GenericRecord] - // check schema - assertEquals(cdcRecord.getSchema, cdcSchema) - if (cdcSupplementalLoggingMode == "cdc_op_key") { - // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) - } else if (cdcSupplementalLoggingMode == "cdc_data_before") { - // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == newHoodieRecords.map(_.getKey.getRecordKey).sorted) - // check before - if (op == HoodieCDCOperation.INSERT) { - assertNull(cdcRecord.get("before")) - } else { - val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcRecord.get("record_key").toString).get - .getData.asInstanceOf[RawTripTestPayload] - val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] - val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] - assertNotEquals(genericRecord.get("begin_lat"), cdcBeforeValue.get("begin_lat")) - } - } else { - val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] - val cdcAfterValue = cdcRecord.get("after").asInstanceOf[GenericRecord] - if (op == HoodieCDCOperation.INSERT) { - // check before - assertNull(cdcBeforeValue) - // check after - val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcAfterValue.get("_row_key").toString).get - .getData.asInstanceOf[RawTripTestPayload] - val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] - assertEquals(genericRecord.get("begin_lat"), cdcAfterValue.get("begin_lat")) - } else { - val payload = newHoodieRecords.find(_.getKey.getRecordKey == cdcAfterValue.get("_row_key").toString).get - .getData.asInstanceOf[RawTripTestPayload] - val genericRecord = payload.getInsertValue(dataSchema).get.asInstanceOf[GenericRecord] - // check before - assertNotEquals(genericRecord.get("begin_lat"), cdcBeforeValue.get("begin_lat")) - // check after - assertEquals(genericRecord.get("begin_lat"), cdcAfterValue.get("begin_lat")) - } - } - } + totalInsertedCnt += 100 + val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() + // all the data is new-coming, it will write out cdc log files. + assertFalse(hasCDCLogFile(instant1)) + val commitTime1 = instant1.getTimestamp + val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) + + // Insert Overwrite Partition Operation + val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 30, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val instant2 = metaClient.reloadActiveTimeline.lastInstant().get() + // the files which keep all the old data will be replaced directly. + // and all the new data will write out some new file groups. + // it will NOT write out cdc log files + assertFalse(hasCDCLogFile(instant2)) + val commitTime2 = instant2.getTimestamp + val insertedCnt2 = 30 + val deletedCnt2 = partitionToCnt(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + val cdcDataOnly2 = cdcDataFrame((commitTime2.toLong - 1).toString) + assertCDCOpCnt(cdcDataOnly2, insertedCnt2, 0, deletedCnt2) + + totalInsertedCnt += insertedCnt2 + totalDeletedCnt += deletedCnt2 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // Drop Partition + spark.emptyDataFrame.write.format("org.apache.hudi") + .options(options) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH) + .mode(SaveMode.Append) + .save(basePath) + val instant3 = metaClient.reloadActiveTimeline.lastInstant().get() + // the files belongs to this partition will be replaced directly. + // it will NOT write out cdc log files. + assertFalse(hasCDCLogFile(instant3)) + val commitTime3 = instant3.getTimestamp + val cntForInstant3 = spark.read.format("hudi").load(basePath).count() + // here we use `commitTime2` to query the change data in commit 3. + // because `commitTime3` is the ts of the clustering operation, not the delete operation. + val cdcDataOnly3 = cdcDataFrame((commitTime3.toLong - 1).toString) + val deletedCnt3 = partitionToCnt(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH) + assertCDCOpCnt(cdcDataOnly3, 0, 0, deletedCnt3) + + totalDeletedCnt += deletedCnt3 + // all the change data in the range [commitTime1, commitTime3] + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) - private def checkCDCDataForDelete(cdcSupplementalLoggingMode: String, - cdcSchema: Schema, - cdcRecords: Seq[IndexedRecord], - deletedKeys: java.util.List[HoodieKey]): Unit = { - val cdcRecord = cdcRecords.head.asInstanceOf[GenericRecord] - // check schema - assertEquals(cdcRecord.getSchema, cdcSchema) - if (cdcSupplementalLoggingMode == "cdc_op_key") { - // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == deletedKeys.map(_.getRecordKey).sorted) - } else if (cdcSupplementalLoggingMode == "cdc_data_before") { - // check record key - assert(cdcRecords.map(_.get(1).toString).sorted == deletedKeys.map(_.getRecordKey).sorted) - } else { - val cdcBeforeValue = cdcRecord.get("before").asInstanceOf[GenericRecord] - val cdcAfterValue = cdcRecord.get("after").asInstanceOf[GenericRecord] - // check before - assert(deletedKeys.exists(_.getRecordKey == cdcBeforeValue.get("_row_key").toString)) - // check after - assertNull(cdcAfterValue) - } + // Upsert Operation + val records4 = recordsToStrings(dataGen.generateUniqueUpdates("000", 50)).toList + val inputDF4 = spark.read.json(spark.sparkContext.parallelize(records4, 2)) + inputDF4.write.format("org.apache.hudi") + .options(options) + .mode(SaveMode.Append) + .save(basePath) + val instant4 = metaClient.reloadActiveTimeline.lastInstant().get() + val commitTime4 = instant4.getTimestamp + val cntForInstant4 = spark.read.format("hudi").load(basePath).count() + val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) + val insertedCnt4 = cntForInstant4 - cntForInstant3 + val updatedCnt4 = 50 - insertedCnt4 + assertCDCOpCnt(cdcDataOnly4, insertedCnt4, updatedCnt4, 0) + + totalInsertedCnt += insertedCnt4 + totalUpdatedCnt += updatedCnt4 + allVisibleCDCData = cdcDataFrame((commitTime1.toLong - 1).toString) + assertCDCOpCnt(allVisibleCDCData, totalInsertedCnt, totalUpdatedCnt, totalDeletedCnt) + + // check both starting and ending are provided + val cdcDataFrom2To3 = cdcDataFrame((commitTime2.toLong - 1).toString, commitTime3) + assertCDCOpCnt(cdcDataFrom2To3, insertedCnt2, 0, deletedCnt2 + deletedCnt3) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala new file mode 100644 index 0000000000000..873d55d64cd89 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala @@ -0,0 +1,214 @@ +/* + * 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.hudi.functional.cdc + +import org.apache.hudi.DataSourceReadOptions +import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.config.HoodieWriteConfig + +import org.apache.spark.sql.QueryTest.checkAnswer +import org.apache.spark.sql.{Column, Dataset, Row, SaveMode} +import org.apache.spark.sql.catalyst.expressions.{Add, If, Literal} +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ + +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.CsvSource + +class TestCDCStreamingSuite extends HoodieCDCTestBase { + + /** + * Here we simulate a more complex streaming data ETL of a real scenario that uses CDC. + * + * there are three tables/data sources: + * -- a streaming data source with three fields userid, country, ts; + * -- a hudi table user_to_country_tbl(userid, country, ts) that enable cdc. + * a streaming can receive the streaming data source and merge into this table; + * -- another hudi table country_to_population_tbl(country, cnt) that not enable cdc; + * and another streaming read the cdc data from user_to_country_tbl and do some aggregation + * and write to country_to_population_tbl. + */ + @ParameterizedTest + @CsvSource(Array("cdc_op_key", "cdc_data_before", "cdc_data_before_after")) + def cdcStreaming(cdcSupplementalLoggingMode: String): Unit = { + val commonOptions = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + "hoodie.bulkinsert.shuffle.parallelism" -> "2", + "hoodie.delete.shuffle.parallelism" -> "1" + ) + + val _spark = spark + import _spark.implicits._ + + val userToCountryTblPath = s"$basePath/user_to_country_table" + val countryToPopulationTblPath = s"$basePath/country_to_population_table" + + // define user_to_country_tbl and enable CDC. + // assume that there already are some records in user_to_country_tbl. + val userToCountryDF = Seq( + (1, "US", "1000"), (2, "US", "1000"), + (3, "China", "1000"), (4, "Singapore", "1000") + ).toDF("userid", "country", "ts") + userToCountryDF.write.format("hudi") + .options(commonOptions) + .option(HoodieTableConfig.CDC_ENABLED.key, "true") + .option(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key, cdcSupplementalLoggingMode) + .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "userid") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") + .option(HoodieWriteConfig.TBL_NAME.key, "user_to_country") + .save(userToCountryTblPath) + + // define country_to_population_tbl as a normal hudi table. + // assume that there already are some records in country_to_population_tbl. + val countryToPopulationDF = Seq( + ("US", 200, "1000"), ("China", 50, "1000"), ("Singapore", 20, "1000") + ).toDF("country", "population", "ts") + countryToPopulationDF.write.format("hudi") + .options(commonOptions) + .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "country") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") + .option(HoodieWriteConfig.TBL_NAME.key, "country_to_population") + .save(countryToPopulationTblPath) + + val hadoopConf = spark.sessionState.newHadoopConf() + val userToCountryMetaClient = HoodieTableMetaClient.builder() + .setBasePath(userToCountryTblPath) + .setConf(hadoopConf) + .build() + + val inputData = new MemoryStream[(Int, String, String)](100, spark.sqlContext) + val df = inputData.toDS().toDF("userid", "country", "ts") + // stream1: from upstream data source to user_to_country_tbl + val stream1 = df.writeStream + .format("hudi") + .foreachBatch { (batch: Dataset[Row], id: Long) => + batch.write.format("hudi") + .options(commonOptions) + .option(HoodieTableConfig.CDC_ENABLED.key, "true") + .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "userid") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") + .option(HoodieWriteConfig.TBL_NAME.key, "user_to_country") + .mode(SaveMode.Append) + .save(userToCountryTblPath) + } + .start() + + // stream2: extract the change data from user_to_country_tbl and merge into country_to_population_tbl + val dec = typedLit(-1).expr + val inc = typedLit(1).expr + val zero = typedLit(0).expr + val beforeCntExpr = If(isnull(col("bcountry")).expr, zero, dec) + val afterCntExpr = If(isnull(col("acountry")).expr, zero, inc) + val stream2 = spark.readStream.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.INCREMENTAL_FORMAT.key, DataSourceReadOptions.INCREMENTAL_FORMAT_CDC_VAL) + .load(userToCountryTblPath) + .writeStream + .format("hudi") + .foreachBatch { (batch: Dataset[Row], id: Long) => + val current = spark.read.format("hudi").load(countryToPopulationTblPath) + batch + // extract the value of the country field from the`before` field and `after` field. + .select( + get_json_object(col("before"), "$.country").as("bcountry"), + get_json_object(col("after"), "$.country").as("acountry"), + get_json_object(col("after"), "$.ts").as("ts") + ) + // aggregate data by country, get the delta change about the population of a country. + .withColumn("bcnt", new Column(beforeCntExpr)).withColumn("acnt", new Column(afterCntExpr)) + .select( + explode(array(Array( + struct(col("bcountry").as("country"), col("bcnt").as("cnt"), col("ts")), + struct(col("acountry").as("country"), col("acnt").as("cnt"), col("ts"))): _*)) + ) + .select(col("col.country").as("country"), col("col.cnt").as("cnt"), col("col.ts").as("ts")) + .where("country is not null").groupBy("country") + .agg(("cnt" -> "sum"), ("ts" -> "max")) + // join with the current data of country_to_population_tbl, get the current population for each of country. + .join(current, Seq("country"), "left") + .select( + col("country"), + new Column( + Add(col("sum(cnt)").expr, If(isnull(col("population")).expr, Literal(0), col("population").expr))).as("population"), + col("max(ts)").as("ts") + ) + .write.format("hudi") + .options(commonOptions) + .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "country") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts") + .option(HoodieWriteConfig.TBL_NAME.key, "country_to_population") + .mode(SaveMode.Append) + .save(countryToPopulationTblPath) + } + .start() + + // fake upstream batch1 + inputData.addData(Seq((3, "US", "1100"), (4, "US", "1100"), (5, "US", "1100"))) + stream1.processAllAvailable() + stream2.processAllAvailable() + + // check the change data about user_to_country_tbl for batch1 + val detailOutput1 = spark.read.format("hudi").load(userToCountryTblPath) + assert(detailOutput1.where("country = 'US'").count() == 5) + val ucTs1 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.getTimestamp + val ucDdcData1 = cdcDataFrame(userToCountryTblPath, (ucTs1.toLong - 1).toString, null) + ucDdcData1.show(false) + assertCDCOpCnt(ucDdcData1, 1, 2, 0) + + // check the final data of country_to_population_tbl for batch1 + val countryRes1 = Seq( + Row("China", 50), + Row("Singapore", 20), + Row("US", 205) + ) + var currentCP = spark.read.format("hudi") + .load(countryToPopulationTblPath) + .select("country", "population") + .sort("country") + checkAnswer(currentCP, countryRes1) + + // fake upstream batch2 + inputData.addData(Seq((3, "Singapore", "1200"), (7, "Canada", "1200"), (8, "Singapore", "1200"))) + stream1.processAllAvailable() + stream2.processAllAvailable() + + // check the change data about user_to_country_tbl for batch2 + val ts2 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.getTimestamp + val cdcData2 = cdcDataFrame(userToCountryTblPath, (ts2.toLong - 1).toString, null) + cdcData2.show(false) + assertCDCOpCnt(cdcData2, 2, 1, 0) + + // check the final data of country_to_population_tbl for batch2 + val countryRes2 = Seq( + Row("Canada", 1), + Row("China", 50), + Row("Singapore", 22), + Row("US", 204) + ) + currentCP = spark.read.format("hudi") + .load(countryToPopulationTblPath) + .select("country", "population") + .sort("country") + checkAnswer(currentCP, countryRes2) + + stream1.stop() + stream2.stop() + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCDCForSparkSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCDCForSparkSQL.scala new file mode 100644 index 0000000000000..60aa3c3e0774f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCDCForSparkSQL.scala @@ -0,0 +1,255 @@ +/* + * 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.spark.sql.hudi + +import org.apache.hudi.DataSourceReadOptions._ +import org.apache.hudi.common.table.HoodieTableMetaClient + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ + +import org.junit.jupiter.api.Assertions.assertEquals + +class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { + + def cdcDataFrame(basePath: String, startingTs: Long, endingTs: Option[Long] = None): DataFrame = { + val reader = spark.read.format("hudi") + .option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(INCREMENTAL_FORMAT.key, INCREMENTAL_FORMAT_CDC_VAL) + .option(BEGIN_INSTANTTIME.key, startingTs.toString) + endingTs.foreach { ts => + reader.option(END_INSTANTTIME.key, ts.toString) + } + reader.load(basePath) + } + + def assertCDCOpCnt(cdcData: DataFrame, expectedInsertCnt: Long, + expectedUpdateCnt: Long, expectedDeletedCnt: Long): Unit = { + assertEquals(expectedInsertCnt, cdcData.where("op = 'i'").count()) + assertEquals(expectedUpdateCnt, cdcData.where("op = 'u'").count()) + assertEquals(expectedDeletedCnt, cdcData.where("op = 'd'").count()) + } + + /** + * Test CDC in cases that it's a COW/MOR non--partitioned table and `cdcSupplementalLoggingMode` is true or not. + */ + test("Test Non-Partitioned Hoodie Table") { + val databaseName = "hudi_database" + spark.sql(s"create database if not exists $databaseName") + spark.sql(s"use $databaseName") + + Seq("cow", "mor").foreach { tableType => + Seq("cdc_op_key", "cdc_data_before").foreach { cdcSupplementalLoggingMode => + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long + | ) using hudi + | tblproperties ( + | 'primaryKey' = 'id', + | 'preCombineField' = 'ts', + | 'hoodie.table.cdc.enabled' = 'true', + | 'hoodie.table.cdc.supplemental.logging.mode' = '$cdcSupplementalLoggingMode', + | type = '$tableType' + | ) + | location '$basePath' + """.stripMargin) + + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(basePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + + spark.sql(s"insert into $tableName values (1, 'a1', 11, 1000), (2, 'a2', 12, 1000), (3, 'a3', 13, 1000)") + val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly1 = cdcDataFrame(basePath, commitTime1.toLong - 1) + cdcDataOnly1.show(false) + assertCDCOpCnt(cdcDataOnly1, 3, 0, 0) + + spark.sql(s"insert into $tableName values (1, 'a1_v2', 11, 1100)") + val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly2 = cdcDataFrame(basePath, commitTime2.toLong - 1) + cdcDataOnly2.show(false) + assertCDCOpCnt(cdcDataOnly2, 0, 1, 0) + + // Check the details + val originSchema = spark.read.format("hudi").load(basePath).schema + val change2 = cdcDataOnly2.select( + col("op"), + from_json(col("before"), originSchema).as("before"), + from_json(col("after"), originSchema).as("after") + ).select( + col("op"), + col("after.id"), + col("before.name"), + col("before.price"), + col("after.name"), + col("after.price") + ).collect() + checkAnswer(change2)(Array("u", 1, "a1", 11, "a1_v2", 11)) + + spark.sql(s"update $tableName set name = 'a2_v2', ts = 1200 where id = 2") + val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly3 = cdcDataFrame(basePath, commitTime3.toLong - 1) + cdcDataOnly3.show(false) + assertCDCOpCnt(cdcDataOnly3, 0, 1, 0) + + spark.sql(s"delete from $tableName where id = 3") + val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly4 = cdcDataFrame(basePath, commitTime4.toLong - 1) + cdcDataOnly4.show(false) + assertCDCOpCnt(cdcDataOnly4, 0, 0, 1) + + spark.sql( + s""" + | merge into $tableName + | using ( + | select * from ( + | select 1 as id, 'a1_v3' as name, cast(11 as double) as price, cast(1300 as long) as ts + | union all + | select 4 as id, 'a4' as name, cast(14 as double) as price, cast(1300 as long) as ts + | ) + | ) s0 + | on s0.id = $tableName.id + | when matched then update set id = s0.id, name = s0.name, price = s0.price, ts = s0.ts + | when not matched then insert * + """.stripMargin) + val commitTime5 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly5 = cdcDataFrame(basePath, commitTime5.toLong - 1) + cdcDataOnly5.show(false) + assertCDCOpCnt(cdcDataOnly5, 1, 1, 0) + + // Check the details + val change5 = cdcDataOnly5.select( + col("op"), + from_json(col("before"), originSchema).as("before"), + from_json(col("after"), originSchema).as("after") + ).select( + col("op"), + col("after.id"), + col("before.name"), + col("before.price"), + col("after.name"), + col("after.price") + ).collect() + checkAnswer(change5.sortBy(_.getInt(1)))( + Array("u", 1, "a1_v2", 11, "a1_v3", 11), + Array("i", 4, null, null, "a4", 14) + ) + + val totalCdcData = cdcDataFrame(basePath, commitTime1.toLong - 1) + assertCDCOpCnt(totalCdcData, 4, 3, 1) + } + } + } + } + + /** + * Test CDC in cases that it's a COW/MOR partitioned table and `cdcSupplementalLoggingMode` is true or not. + */ + test("Test Partitioned Hoodie Table") { + val databaseName = "hudi_database" + spark.sql(s"create database if not exists $databaseName") + spark.sql(s"use $databaseName") + + Seq("cow", "mor").foreach { tableType => + Seq("cdc_op_key", "cdc_data_before").foreach { cdcSupplementalLoggingMode => + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | pt string + | ) using hudi + | partitioned by (pt) + | tblproperties ( + | 'primaryKey' = 'id', + | 'preCombineField' = 'ts', + | 'hoodie.table.cdc.enabled' = 'true', + | 'hoodie.table.cdc.supplemental.logging.mode' = '$cdcSupplementalLoggingMode', + | 'type' = '$tableType' + | ) + | location '$basePath' + """.stripMargin) + + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(basePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + + spark.sql( + s""" + | insert into $tableName values + | (1, 'a1', 11, 1000, '2021'), + | (2, 'a2', 12, 1000, '2022'), + | (3, 'a3', 13, 1000, '2022') + """.stripMargin) + val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly1 = cdcDataFrame(basePath, commitTime1.toLong - 1) + cdcDataOnly1.show(false) + assertCDCOpCnt(cdcDataOnly1, 3, 0, 0) + + spark.sql(s"insert overwrite table $tableName partition (pt = '2021') values (1, 'a1_v2', 11, 1100)") + val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly2 = cdcDataFrame(basePath, commitTime2.toLong - 1) + cdcDataOnly2.show(false) + assertCDCOpCnt(cdcDataOnly2, 1, 0, 1) + + spark.sql(s"update $tableName set name = 'a2_v2', ts = 1200 where id = 2") + val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly3 = cdcDataFrame(basePath, commitTime3.toLong - 1) + cdcDataOnly3.show(false) + assertCDCOpCnt(cdcDataOnly3, 0, 1, 0) + + spark.sql( + s""" + | merge into $tableName + | using ( + | select * from ( + | select 1 as id, 'a1_v3' as name, cast(11 as double) as price, cast(1300 as long) as ts, "2021" as pt + | union all + | select 4 as id, 'a4' as name, cast(14 as double) as price, cast(1300 as long) as ts, "2022" as pt + | ) + | ) s0 + | on s0.id = $tableName.id + | when matched then update set id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, pt = s0.pt + | when not matched then insert * + """.stripMargin) + val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val cdcDataOnly4 = cdcDataFrame(basePath, commitTime4.toLong - 1) + cdcDataOnly4.show(false) + assertCDCOpCnt(cdcDataOnly4, 1, 1, 0) + + val totalCdcData = cdcDataFrame(basePath, commitTime1.toLong - 1) + assertCDCOpCnt(totalCdcData, 5, 2, 1) + } + } + } + } +}