Skip to content

Commit 32f45f0

Browse files
authored
[HUDI-5634] Rename CDC related classes (#7410)
1 parent e6c0bd6 commit 32f45f0

5 files changed

Lines changed: 49 additions & 52 deletions

File tree

hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -56,11 +56,11 @@
5656
import java.util.Set;
5757
import java.util.stream.Collectors;
5858

59-
import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.AS_IS;
60-
import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_DELETE;
61-
import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_INSERT;
62-
import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.LOG_FILE;
63-
import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.REPLACE_COMMIT;
59+
import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.AS_IS;
60+
import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.BASE_FILE_DELETE;
61+
import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.BASE_FILE_INSERT;
62+
import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.LOG_FILE;
63+
import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.REPLACE_COMMIT;
6464
import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before_after;
6565
import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.op_key_only;
6666
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;

hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -33,16 +33,16 @@
3333
* This contains all the information that retrieve the change data at a single file group and
3434
* at a single commit.
3535
* <p>
36-
* For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_INSERT}, `cdcFile` is a current version of
36+
* For `cdcInferCase` = {@link HoodieCDCInferenceCase#BASE_FILE_INSERT}, `cdcFile` is a current version of
3737
* the base file in the group, and `beforeFileSlice` is None.
38-
* For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_DELETE}, `cdcFile` is null,
38+
* For `cdcInferCase` = {@link HoodieCDCInferenceCase#BASE_FILE_DELETE}, `cdcFile` is null,
3939
* `beforeFileSlice` is the previous version of the base file in the group.
40-
* For `cdcInferCase` = {@link HoodieCDCInferCase#AS_IS}, `cdcFile` is a log file with cdc blocks.
40+
* For `cdcInferCase` = {@link HoodieCDCInferenceCase#AS_IS}, `cdcFile` is a log file with cdc blocks.
4141
* when enable the supplemental logging, both `beforeFileSlice` and `afterFileSlice` are None,
4242
* otherwise these two are the previous and current version of the base file.
43-
* For `cdcInferCase` = {@link HoodieCDCInferCase#LOG_FILE}, `cdcFile` is a normal log file and
43+
* For `cdcInferCase` = {@link HoodieCDCInferenceCase#LOG_FILE}, `cdcFile` is a normal log file and
4444
* `beforeFileSlice` is the previous version of the file slice.
45-
* For `cdcInferCase` = {@link HoodieCDCInferCase#REPLACE_COMMIT}, `cdcFile` is null,
45+
* For `cdcInferCase` = {@link HoodieCDCInferenceCase#REPLACE_COMMIT}, `cdcFile` is null,
4646
* `beforeFileSlice` is the current version of the file slice.
4747
*/
4848
public class HoodieCDCFileSplit implements Serializable, Comparable<HoodieCDCFileSplit> {
@@ -54,7 +54,7 @@ public class HoodieCDCFileSplit implements Serializable, Comparable<HoodieCDCFil
5454
/**
5555
* Flag that decides to how to retrieve the change data. More details see: `HoodieCDCLogicalFileType`.
5656
*/
57-
private final HoodieCDCInferCase cdcInferCase;
57+
private final HoodieCDCInferenceCase cdcInferCase;
5858

5959
/**
6060
* The file that the change data can be parsed from.
@@ -71,17 +71,17 @@ public class HoodieCDCFileSplit implements Serializable, Comparable<HoodieCDCFil
7171
*/
7272
private final Option<FileSlice> afterFileSlice;
7373

74-
public HoodieCDCFileSplit(String instant, HoodieCDCInferCase cdcInferCase, String cdcFile) {
74+
public HoodieCDCFileSplit(String instant, HoodieCDCInferenceCase cdcInferCase, String cdcFile) {
7575
this(instant, cdcInferCase, cdcFile, Option.empty(), Option.empty());
7676
}
7777

78-
public HoodieCDCFileSplit(String instant, HoodieCDCInferCase cdcInferCase, Collection<String> cdcFiles) {
78+
public HoodieCDCFileSplit(String instant, HoodieCDCInferenceCase cdcInferCase, Collection<String> cdcFiles) {
7979
this(instant, cdcInferCase, cdcFiles, Option.empty(), Option.empty());
8080
}
8181

8282
public HoodieCDCFileSplit(
8383
String instant,
84-
HoodieCDCInferCase cdcInferCase,
84+
HoodieCDCInferenceCase cdcInferCase,
8585
String cdcFile,
8686
Option<FileSlice> beforeFileSlice,
8787
Option<FileSlice> afterFileSlice) {
@@ -90,7 +90,7 @@ public HoodieCDCFileSplit(
9090

9191
public HoodieCDCFileSplit(
9292
String instant,
93-
HoodieCDCInferCase cdcInferCase,
93+
HoodieCDCInferenceCase cdcInferCase,
9494
Collection<String> cdcFiles,
9595
Option<FileSlice> beforeFileSlice,
9696
Option<FileSlice> afterFileSlice) {
@@ -106,7 +106,7 @@ public String getInstant() {
106106
return this.instant;
107107
}
108108

109-
public HoodieCDCInferCase getCdcInferCase() {
109+
public HoodieCDCInferenceCase getCdcInferCase() {
110110
return this.cdcInferCase;
111111
}
112112

hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java renamed to hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferenceCase.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@
4242
* file is new-coming, so we can load this, mark all the records with `i`, and treat them as
4343
* the value of `after`. The value of `before` for each record is null.
4444
*
45-
* BASE_FILE_INSERT:
45+
* BASE_FILE_DELETE:
4646
* For this type, there must be an empty file at the current instant, but a non-empty base file
4747
* at the previous instant. First we find this base file that has the same file group and belongs
4848
* to the previous instant. Then load this, mark all the records with `d`, and treat them as
@@ -67,7 +67,7 @@
6767
* a whole file group. First we find this file group. Then load this, mark all the records with
6868
* `d`, and treat them as the value of `before`. The value of `after` for each record is null.
6969
*/
70-
public enum HoodieCDCInferCase {
70+
public enum HoodieCDCInferenceCase {
7171

7272
AS_IS,
7373
BASE_FILE_INSERT,

hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/HoodieCDCRDD.scala

Lines changed: 30 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.hudi.avro.HoodieAvroUtils
3232
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
3333
import org.apache.hudi.common.model._
3434
import org.apache.hudi.common.table.HoodieTableMetaClient
35-
import org.apache.hudi.common.table.cdc.HoodieCDCInferCase._
35+
import org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase._
3636
import org.apache.hudi.common.table.cdc.HoodieCDCOperation._
3737
import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode._
3838
import org.apache.hudi.common.table.cdc.{HoodieCDCFileSplit, HoodieCDCUtils}
@@ -81,10 +81,11 @@ class HoodieCDCRDD(
8181
originTableSchema: HoodieTableSchema,
8282
cdcSchema: StructType,
8383
requiredCdcSchema: StructType,
84-
changes: Array[HoodieCDCFileGroupSplit])
84+
@transient changes: Array[HoodieCDCFileGroupSplit])
8585
extends RDD[InternalRow](spark.sparkContext, Nil) with HoodieUnsafeRDD {
8686

87-
@transient private val hadoopConf = spark.sparkContext.hadoopConfiguration
87+
@transient
88+
private val hadoopConf = spark.sparkContext.hadoopConfiguration
8889

8990
private val confBroadcast = spark.sparkContext.broadcast(new SerializableWritable(hadoopConf))
9091

@@ -118,7 +119,7 @@ class HoodieCDCRDD(
118119

119120
private lazy val fs = metaClient.getFs.getFileSystem
120121

121-
private lazy val conf = new Configuration(confBroadcast.value.value)
122+
private lazy val conf = confBroadcast.value.value
122123

123124
private lazy val basePath = metaClient.getBasePathV2
124125

@@ -127,11 +128,7 @@ class HoodieCDCRDD(
127128
private lazy val populateMetaFields = tableConfig.populateMetaFields()
128129

129130
private lazy val keyGenerator = {
130-
val props = new TypedProperties()
131-
props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, tableConfig.getKeyGeneratorClassName)
132-
props.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key, tableConfig.getRecordKeyFieldProp)
133-
props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key, tableConfig.getPartitionFieldProp)
134-
HoodieSparkKeyGeneratorFactory.createKeyGenerator(props)
131+
HoodieSparkKeyGeneratorFactory.createKeyGenerator(tableConfig.getProps())
135132
}
136133

137134
private lazy val recordKeyField: String = if (populateMetaFields) {
@@ -202,7 +199,7 @@ class HoodieCDCRDD(
202199
private var currentInstant: String = _
203200

204201
// The change file that is currently being processed
205-
private var currentChangeFile: HoodieCDCFileSplit = _
202+
private var currentCDCFileSplit: HoodieCDCFileSplit = _
206203

207204
/**
208205
* Two cases will use this to iterator the records:
@@ -258,10 +255,10 @@ class HoodieCDCRDD(
258255
if (needLoadNextFile) {
259256
loadCdcFile()
260257
}
261-
if (currentChangeFile == null) {
258+
if (currentCDCFileSplit == null) {
262259
false
263260
} else {
264-
currentChangeFile.getCdcInferCase match {
261+
currentCDCFileSplit.getCdcInferCase match {
265262
case BASE_FILE_INSERT | BASE_FILE_DELETE | REPLACE_COMMIT =>
266263
if (recordIter.hasNext && loadNext()) {
267264
true
@@ -292,7 +289,7 @@ class HoodieCDCRDD(
292289

293290
def loadNext(): Boolean = {
294291
var loaded = false
295-
currentChangeFile.getCdcInferCase match {
292+
currentCDCFileSplit.getCdcInferCase match {
296293
case BASE_FILE_INSERT =>
297294
val originRecord = recordIter.next()
298295
recordToLoad.update(3, convertRowToJsonString(originRecord))
@@ -416,48 +413,48 @@ class HoodieCDCRDD(
416413
if (cdcFileIter.hasNext) {
417414
val split = cdcFileIter.next()
418415
currentInstant = split.getInstant
419-
currentChangeFile = split
420-
currentChangeFile.getCdcInferCase match {
416+
currentCDCFileSplit = split
417+
currentCDCFileSplit.getCdcInferCase match {
421418
case BASE_FILE_INSERT =>
422-
assert(currentChangeFile.getCdcFiles != null && currentChangeFile.getCdcFiles.size() == 1)
423-
val absCDCPath = new Path(basePath, currentChangeFile.getCdcFiles.get(0))
419+
assert(currentCDCFileSplit.getCdcFiles != null && currentCDCFileSplit.getCdcFiles.size() == 1)
420+
val absCDCPath = new Path(basePath, currentCDCFileSplit.getCdcFiles.get(0))
424421
val fileStatus = fs.getFileStatus(absCDCPath)
425422
val pf = PartitionedFile(InternalRow.empty, absCDCPath.toUri.toString, 0, fileStatus.getLen)
426423
recordIter = parquetReader(pf)
427424
case BASE_FILE_DELETE =>
428-
assert(currentChangeFile.getBeforeFileSlice.isPresent)
429-
recordIter = loadFileSlice(currentChangeFile.getBeforeFileSlice.get)
425+
assert(currentCDCFileSplit.getBeforeFileSlice.isPresent)
426+
recordIter = loadFileSlice(currentCDCFileSplit.getBeforeFileSlice.get)
430427
case LOG_FILE =>
431-
assert(currentChangeFile.getCdcFiles != null && currentChangeFile.getCdcFiles.size() == 1
432-
&& currentChangeFile.getBeforeFileSlice.isPresent)
433-
loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get)
434-
val absLogPath = new Path(basePath, currentChangeFile.getCdcFiles.get(0))
428+
assert(currentCDCFileSplit.getCdcFiles != null && currentCDCFileSplit.getCdcFiles.size() == 1
429+
&& currentCDCFileSplit.getBeforeFileSlice.isPresent)
430+
loadBeforeFileSliceIfNeeded(currentCDCFileSplit.getBeforeFileSlice.get)
431+
val absLogPath = new Path(basePath, currentCDCFileSplit.getCdcFiles.get(0))
435432
val morSplit = HoodieMergeOnReadFileSplit(None, List(new HoodieLogFile(fs.getFileStatus(absLogPath))))
436433
val logFileIterator = new LogFileIterator(morSplit, originTableSchema, originTableSchema, tableState, conf)
437434
logRecordIter = logFileIterator.logRecordsPairIterator
438435
case AS_IS =>
439-
assert(currentChangeFile.getCdcFiles != null && !currentChangeFile.getCdcFiles.isEmpty)
436+
assert(currentCDCFileSplit.getCdcFiles != null && !currentCDCFileSplit.getCdcFiles.isEmpty)
440437
// load beforeFileSlice to beforeImageRecords
441-
if (currentChangeFile.getBeforeFileSlice.isPresent) {
442-
loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get)
438+
if (currentCDCFileSplit.getBeforeFileSlice.isPresent) {
439+
loadBeforeFileSliceIfNeeded(currentCDCFileSplit.getBeforeFileSlice.get)
443440
}
444441
// load afterFileSlice to afterImageRecords
445-
if (currentChangeFile.getAfterFileSlice.isPresent) {
446-
val iter = loadFileSlice(currentChangeFile.getAfterFileSlice.get())
442+
if (currentCDCFileSplit.getAfterFileSlice.isPresent) {
443+
val iter = loadFileSlice(currentCDCFileSplit.getAfterFileSlice.get())
447444
afterImageRecords = mutable.Map.empty
448445
iter.foreach { row =>
449446
val key = getRecordKey(row)
450447
afterImageRecords.put(key, row.copy())
451448
}
452449
}
453450

454-
val cdcLogFiles = currentChangeFile.getCdcFiles.asScala.map { cdcFile =>
451+
val cdcLogFiles = currentCDCFileSplit.getCdcFiles.asScala.map { cdcFile =>
455452
new HoodieLogFile(fs.getFileStatus(new Path(basePath, cdcFile)))
456453
}.toArray
457454
cdcLogRecordIterator = new HoodieCDCLogRecordIterator(fs, cdcLogFiles, cdcAvroSchema)
458455
case REPLACE_COMMIT =>
459-
if (currentChangeFile.getBeforeFileSlice.isPresent) {
460-
loadBeforeFileSliceIfNeeded(currentChangeFile.getBeforeFileSlice.get)
456+
if (currentCDCFileSplit.getBeforeFileSlice.isPresent) {
457+
loadBeforeFileSliceIfNeeded(currentCDCFileSplit.getBeforeFileSlice.get)
461458
}
462459
recordIter = beforeImageRecords.values.map { record =>
463460
deserialize(record)
@@ -467,15 +464,15 @@ class HoodieCDCRDD(
467464
resetRecordFormat()
468465
} else {
469466
currentInstant = null
470-
currentChangeFile = null
467+
currentCDCFileSplit = null
471468
}
472469
}
473470

474471
/**
475472
* Initialize the partial fields of the data to be returned in advance to speed up.
476473
*/
477474
private def resetRecordFormat(): Unit = {
478-
recordToLoad = currentChangeFile.getCdcInferCase match {
475+
recordToLoad = currentCDCFileSplit.getCdcInferCase match {
479476
case BASE_FILE_INSERT =>
480477
InternalRow.fromSeq(Array(
481478
CDCRelation.CDC_OPERATION_INSERT, convertToUTF8String(currentInstant),

rfc/rfc-51/rfc-51.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -202,7 +202,7 @@ tblproperties (
202202

203203
### How to infer CDC results
204204

205-
| `HoodieCDCInferCase` | Infer case details | Infer logic | Note |
205+
| `HoodieCDCInferenceCase` | Infer case details | Infer logic | Note |
206206
|----------------------|---------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------|
207207
| `AS_IS` | CDC file written (suffix contains `-cdc`) alongside base files (COW) or log files (MOR) | CDC info will be extracted as is | the read-optimized way to read CDC |
208208
| `BASE_FILE_INSERT` | Base files were written to a new file group | All records (in the current commit): `op=I`, `before=null`, `after=<current value>` | on-the-fly inference |

0 commit comments

Comments
 (0)