Skip to content

Commit e987dbd

Browse files
committed
Fix compile errors
1 parent 8b743f7 commit e987dbd

File tree

10 files changed

+35
-47
lines changed

10 files changed

+35
-47
lines changed

hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919

2020
package org.apache.hudi
2121

22-
import org.apache.avro.Schema
2322
import org.apache.hadoop.conf.Configuration
2423
import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{filterIsSafeForBootstrap, filterIsSafeForPrimaryKey, getAppliedRequiredSchema}
2524
import org.apache.hudi.avro.{AvroSchemaUtils, HoodieAvroUtils}
@@ -34,7 +33,6 @@ import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator
3433
import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader}
3534
import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath}
3635
import org.apache.hudi.util.CloseableInternalRowIterator
37-
import org.apache.parquet.avro.AvroSchemaConverter
3836
import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter
3937
import org.apache.spark.sql.HoodieInternalRowUtils
4038
import org.apache.spark.sql.catalyst.InternalRow
@@ -75,19 +73,19 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR
7573
override def getFileRecordIterator(filePath: StoragePath,
7674
start: Long,
7775
length: Long,
78-
dataSchema: Schema, // dataSchema refers to table schema in most cases(non log file reads).
79-
requiredSchema: Schema,
76+
dataSchema: HoodieSchema, // dataSchema refers to table schema in most cases(non log file reads).
77+
requiredSchema: HoodieSchema,
8078
storage: HoodieStorage): ClosableIterator[InternalRow] = {
81-
val hasRowIndexField = AvroSchemaUtils.containsFieldInSchema(requiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)
79+
val hasRowIndexField = AvroSchemaUtils.containsFieldInSchema(requiredSchema.toAvroSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)
8280
if (hasRowIndexField) {
8381
assert(getRecordContext.supportsParquetRowIndex())
8482
}
85-
val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema)
83+
val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema.toAvroSchema)
8684
val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType, hasRowIndexField)
8785
if (FSUtils.isLogFile(filePath)) {
8886
// NOTE: now only primary key based filtering is supported for log files
8987
new HoodieSparkFileReaderFactory(storage).newParquetFileReader(filePath)
90-
.asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(requiredSchema, readFilters.asJava).asInstanceOf[ClosableIterator[InternalRow]]
88+
.asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(requiredSchema.toAvroSchema, readFilters.asJava).asInstanceOf[ClosableIterator[InternalRow]]
9189
} else {
9290
// partition value is empty because the spark parquet reader will append the partition columns to
9391
// each row if they are given. That is the only usage of the partition values in the reader.
@@ -97,7 +95,7 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR
9795
// Convert Avro dataSchema to Parquet MessageType for timestamp precision conversion
9896
val tableSchemaOpt = if (dataSchema != null) {
9997
val hadoopConf = storage.getConf.unwrapAs(classOf[Configuration])
100-
val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(dataSchema)
98+
val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(dataSchema.toAvroSchema)
10199
org.apache.hudi.common.util.Option.of(parquetSchema)
102100
} else {
103101
org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]()

hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -332,7 +332,7 @@ public static Schema readSchemaFromLogFile(HoodieStorage storage, StoragePath pa
332332
lastBlock = (HoodieDataBlock) block;
333333
}
334334
}
335-
return lastBlock != null ? lastBlock.getSchema() : null;
335+
return lastBlock != null ? lastBlock.getSchema().toAvroSchema() : null;
336336
}
337337
}
338338

hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -819,7 +819,7 @@ private Pair<ClosableIterator<HoodieRecord>, Schema> getRecordsIterator(
819819
.orElse(Function.identity());
820820

821821
Schema schema = schemaEvolutionTransformerOpt.map(Pair::getRight)
822-
.orElseGet(dataBlock::getSchema);
822+
.orElseGet(() -> dataBlock.getSchema().toAvroSchema());
823823

824824
return Pair.of(new CloseableMappingIterator<>(blockRecordsIterator, transformer), schema);
825825
}
@@ -847,7 +847,7 @@ private Option<Pair<Function<HoodieRecord, HoodieRecord>, Schema>> composeEvolve
847847

848848
return Option.of(Pair.of((record) -> {
849849
return record.rewriteRecordWithNewSchema(
850-
dataBlock.getSchema(),
850+
dataBlock.getSchema().toAvroSchema(),
851851
this.hoodieTableMetaClient.getTableConfig().getProps(),
852852
mergedAvroSchema,
853853
Collections.emptyMap());

hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -34,8 +34,6 @@
3434
import org.apache.hudi.common.table.read.buffer.HoodieFileGroupRecordBuffer;
3535
import org.apache.hudi.common.table.timeline.HoodieTimeline;
3636
import org.apache.hudi.common.util.Option;
37-
import org.apache.hudi.common.util.collection.ClosableIterator;
38-
import org.apache.hudi.common.util.collection.Pair;
3937
import org.apache.hudi.exception.HoodieException;
4038
import org.apache.hudi.exception.HoodieIOException;
4139
import org.apache.hudi.internal.schema.InternalSchema;
@@ -655,18 +653,6 @@ public List<String> getValidBlockInstants() {
655653
return validBlockInstants;
656654
}
657655

658-
private Pair<ClosableIterator<T>, Schema> getRecordsIterator(
659-
HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt) throws IOException {
660-
ClosableIterator<T> blockRecordsIterator;
661-
if (keySpecOpt.isPresent()) {
662-
KeySpec keySpec = keySpecOpt.get();
663-
blockRecordsIterator = dataBlock.getEngineRecordIterator(readerContext, keySpec.getKeys(), keySpec.isFullKey());
664-
} else {
665-
blockRecordsIterator = dataBlock.getEngineRecordIterator(readerContext);
666-
}
667-
return Pair.of(blockRecordsIterator, dataBlock.getSchema());
668-
}
669-
670656
/**
671657
* Builder used to build {@code AbstractHoodieLogRecordScanner}.
672658
*/

hudi-common/src/test/java/org/apache/hudi/common/table/read/SchemaHandlerTestBase.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -376,7 +376,8 @@ public UnaryOperator<String> projectRecord(Schema from, Schema to, Map<String, S
376376
}
377377

378378
@Override
379-
public ClosableIterator<String> getFileRecordIterator(StoragePath filePath, long start, long length, HoodieSchema dataSchema, HoodieSchema requiredSchema, HoodieStorage storage) throws IOException {
379+
public ClosableIterator<String> getFileRecordIterator(StoragePath filePath, long start, long length,
380+
HoodieSchema dataSchema, HoodieSchema requiredSchema, HoodieStorage storage) throws IOException {
380381
return null;
381382
}
382383

hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/BaseTestFileGroupRecordBuffer.java

Lines changed: 10 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,9 @@
3232
import org.apache.hudi.common.model.HoodieRecordMerger;
3333
import org.apache.hudi.common.model.HoodieRecordPayload;
3434
import org.apache.hudi.common.model.SerializableIndexedRecord;
35+
import org.apache.hudi.common.schema.HoodieSchema;
36+
import org.apache.hudi.common.schema.HoodieSchemaField;
37+
import org.apache.hudi.common.schema.HoodieSchemaType;
3538
import org.apache.hudi.common.table.HoodieTableConfig;
3639
import org.apache.hudi.common.table.HoodieTableMetaClient;
3740
import org.apache.hudi.common.table.read.BufferedRecord;
@@ -70,14 +73,14 @@
7073

7174
public class BaseTestFileGroupRecordBuffer {
7275

73-
protected static final Schema SCHEMA = Schema.createRecord("test_record", null, "namespace", false,
76+
protected static final HoodieSchema SCHEMA = HoodieSchema.createRecord("test_record", "namespace", null,
7477
Arrays.asList(
75-
new Schema.Field("record_key", Schema.create(Schema.Type.STRING)),
76-
new Schema.Field("counter", Schema.create(Schema.Type.INT)),
77-
new Schema.Field("ts", Schema.create(Schema.Type.LONG))));
78+
HoodieSchemaField.of("record_key", HoodieSchema.create(HoodieSchemaType.STRING)),
79+
HoodieSchemaField.of("counter", HoodieSchema.create(HoodieSchemaType.INT)),
80+
HoodieSchemaField.of("ts", HoodieSchema.create(HoodieSchemaType.LONG))));
7881

7982
protected static GenericRecord createTestRecord(String recordKey, int counter, long ts) {
80-
GenericRecord record = new GenericData.Record(SCHEMA);
83+
GenericRecord record = new GenericData.Record(SCHEMA.toAvroSchema());
8184
record.put("record_key", recordKey);
8285
record.put("counter", counter);
8386
record.put("ts", ts);
@@ -114,9 +117,9 @@ protected static KeyBasedFileGroupRecordBuffer<IndexedRecord> buildKeyBasedFileG
114117
props.setProperty(DELETE_MARKER, markerKeyValue.getRight());
115118
});
116119
FileGroupReaderSchemaHandler<IndexedRecord> fileGroupReaderSchemaHandler = mock(FileGroupReaderSchemaHandler.class);
117-
when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA);
120+
when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA.toAvroSchema());
118121
when(fileGroupReaderSchemaHandler.getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema());
119-
when(fileGroupReaderSchemaHandler.getDeleteContext()).thenReturn(new DeleteContext(props, SCHEMA));
122+
when(fileGroupReaderSchemaHandler.getDeleteContext()).thenReturn(new DeleteContext(props, SCHEMA.toAvroSchema()));
120123
readerContext.setSchemaHandler(fileGroupReaderSchemaHandler);
121124
return buildKeyBasedFileGroupRecordBuffer(readerContext, tableConfig, readStats, recordMerger, recordMergeMode, orderingFieldNames, props,
122125
Option.empty());

hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestFileGroupRecordBufferLoader.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -70,8 +70,8 @@ public void testDefaultFileGroupBufferRecordLoader(String fileGroupRecordBufferT
7070
HoodieReaderContext<IndexedRecord> readerContext = new HoodieAvroReaderContext(storageConfiguration, tableConfig, Option.empty(), Option.empty());
7171
readerContext.initRecordMerger(new TypedProperties());
7272
FileGroupReaderSchemaHandler<IndexedRecord> fileGroupReaderSchemaHandler = mock(FileGroupReaderSchemaHandler.class);
73-
when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA);
74-
when(fileGroupReaderSchemaHandler.getRequestedSchema()).thenReturn(SCHEMA);
73+
when(fileGroupReaderSchemaHandler.getRequiredSchema()).thenReturn(SCHEMA.toAvroSchema());
74+
when(fileGroupReaderSchemaHandler.getRequestedSchema()).thenReturn(SCHEMA.toAvroSchema());
7575
when(fileGroupReaderSchemaHandler.getInternalSchema()).thenReturn(InternalSchema.getEmptyInternalSchema());
7676
DeleteContext deleteContext = mock(DeleteContext.class);
7777
when(deleteContext.getCustomDeleteMarkerKeyValue()).thenReturn(Option.empty());

hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestKeyBasedFileGroupRecordBuffer.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -154,7 +154,7 @@ void readWithEventTimeOrderingWithRecords() throws IOException {
154154
readerContext.setHasLogFiles(false);
155155
readerContext.setHasBootstrapBaseFile(false);
156156
readerContext.initRecordMerger(properties);
157-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
157+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
158158
properties, mock(HoodieTableMetaClient.class));
159159
readerContext.setSchemaHandler(schemaHandler);
160160
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7));
@@ -222,7 +222,7 @@ void readWithCommitTimeOrderingWithRecords() throws IOException {
222222
readerContext.setHasLogFiles(false);
223223
readerContext.setHasBootstrapBaseFile(false);
224224
readerContext.initRecordMerger(properties);
225-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
225+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
226226
properties, mock(HoodieTableMetaClient.class));
227227
readerContext.setSchemaHandler(schemaHandler);
228228
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update,
@@ -302,7 +302,7 @@ void readWithCustomPayloadWithRecords() throws IOException {
302302
readerContext.setHasLogFiles(false);
303303
readerContext.setHasBootstrapBaseFile(false);
304304
readerContext.initRecordMerger(properties);
305-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
305+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
306306
properties, mock(HoodieTableMetaClient.class));
307307
readerContext.setSchemaHandler(schemaHandler);
308308
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate));
@@ -377,7 +377,7 @@ void readWithCustomMergerWithRecords() throws IOException {
377377
readerContext.setHasLogFiles(false);
378378
readerContext.setHasBootstrapBaseFile(false);
379379
readerContext.initRecordMerger(properties);
380-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
380+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
381381
properties, mock(HoodieTableMetaClient.class));
382382
readerContext.setSchemaHandler(schemaHandler);
383383
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate));

hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestSortedKeyBasedFileGroupRecordBuffer.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ void readBaseFileAndLogFile() throws IOException {
9393
fileGroupRecordBuffer.setBaseFileIterator(ClosableIterator.wrap(Arrays.asList(testRecord2, testRecord3, testRecord5).iterator()));
9494

9595
HoodieDataBlock dataBlock = mock(HoodieDataBlock.class);
96-
when(dataBlock.getSchema()).thenReturn(HoodieTestDataGenerator.AVRO_SCHEMA);
96+
when(dataBlock.getSchema()).thenReturn(HoodieTestDataGenerator.HOODIE_SCHEMA);
9797
when(dataBlock.getEngineRecordIterator(mockReaderContext)).thenReturn(
9898
ClosableIterator.wrap(Arrays.asList(testRecord6, testRecord4, testRecord1, testRecord6Update, testRecord2Update).iterator()));
9999

@@ -124,7 +124,7 @@ void readWithStreamingRecordBufferLoaderAndEventTimeOrdering() throws IOExceptio
124124
HoodieReaderContext<IndexedRecord> readerContext = new HoodieAvroReaderContext(storageConfiguration, tableConfig, Option.empty(), Option.empty());
125125
readerContext.setHasLogFiles(false);
126126
readerContext.setHasBootstrapBaseFile(false);
127-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
127+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
128128
properties, mock(HoodieTableMetaClient.class));
129129
readerContext.setSchemaHandler(schemaHandler);
130130
readerContext.initRecordMerger(properties);
@@ -167,11 +167,11 @@ void readLogFiles() throws IOException {
167167
fileGroupRecordBuffer.setBaseFileIterator(ClosableIterator.wrap(Collections.emptyIterator()));
168168

169169
HoodieDataBlock dataBlock1 = mock(HoodieDataBlock.class);
170-
when(dataBlock1.getSchema()).thenReturn(HoodieTestDataGenerator.AVRO_SCHEMA);
170+
when(dataBlock1.getSchema()).thenReturn(HoodieTestDataGenerator.HOODIE_SCHEMA);
171171
when(dataBlock1.getEngineRecordIterator(mockReaderContext)).thenReturn(ClosableIterator.wrap(Arrays.asList(testRecord6, testRecord4, testRecord6Update, testRecord2).iterator()));
172172

173173
HoodieDataBlock dataBlock2 = mock(HoodieDataBlock.class);
174-
when(dataBlock2.getSchema()).thenReturn(HoodieTestDataGenerator.AVRO_SCHEMA);
174+
when(dataBlock2.getSchema()).thenReturn(HoodieTestDataGenerator.HOODIE_SCHEMA);
175175
when(dataBlock2.getEngineRecordIterator(mockReaderContext)).thenReturn(ClosableIterator.wrap(Arrays.asList(testRecord2Update, testRecord5, testRecord3, testRecord1).iterator()));
176176

177177
HoodieDeleteBlock deleteBlock = mock(HoodieDeleteBlock.class);

hudi-common/src/test/java/org/apache/hudi/common/table/read/buffer/TestStreamingKeyBasedFileGroupRecordBuffer.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ void readWithEventTimeOrdering() throws IOException {
8484
readerContext.setHasLogFiles(false);
8585
readerContext.setHasBootstrapBaseFile(false);
8686
readerContext.initRecordMerger(properties);
87-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
87+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
8888
properties, mock(HoodieTableMetaClient.class));
8989
readerContext.setSchemaHandler(schemaHandler);
9090
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7));
@@ -122,7 +122,7 @@ void readWithCommitTimeOrdering() throws IOException {
122122
readerContext.setHasLogFiles(false);
123123
readerContext.setHasBootstrapBaseFile(false);
124124
readerContext.initRecordMerger(properties);
125-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
125+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
126126
properties, mock(HoodieTableMetaClient.class));
127127
readerContext.setSchemaHandler(schemaHandler);
128128
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate, testRecord7));
@@ -162,7 +162,7 @@ void readWithCustomPayload() throws IOException {
162162
readerContext.setHasLogFiles(false);
163163
readerContext.setHasBootstrapBaseFile(false);
164164
readerContext.initRecordMerger(properties);
165-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
165+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
166166
properties, mock(HoodieTableMetaClient.class));
167167
readerContext.setSchemaHandler(schemaHandler);
168168
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate));
@@ -200,7 +200,7 @@ void readWithCustomMergerWithRecords() throws IOException {
200200
readerContext.setHasLogFiles(false);
201201
readerContext.setHasBootstrapBaseFile(false);
202202
readerContext.initRecordMerger(properties);
203-
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA, SCHEMA, Option.empty(),
203+
FileGroupReaderSchemaHandler schemaHandler = new FileGroupReaderSchemaHandler(readerContext, SCHEMA.toAvroSchema(), SCHEMA.toAvroSchema(), Option.empty(),
204204
properties, mock(HoodieTableMetaClient.class));
205205
readerContext.setSchemaHandler(schemaHandler);
206206
List<HoodieRecord> inputRecords = convertToHoodieRecordsList(Arrays.asList(testRecord1UpdateWithSameTime, testRecord2Update, testRecord3Update, testRecord4EarlierUpdate));

0 commit comments

Comments
 (0)