forked from apache/hudi
-
Notifications
You must be signed in to change notification settings - Fork 1
/
TestHoodieRealtimeRecordReader.java
836 lines (753 loc) · 40.4 KB
/
TestHoodieRealtimeRecordReader.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
/*
* 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.hadoop.realtime;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.BaseFileWithLogsSplit;
import org.apache.hudi.hadoop.PathWithLogFilePath;
import org.apache.hudi.hadoop.RealtimeFileStatus;
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.BooleanWritable;
import org.apache.hadoop.io.DoubleWritable;
import org.apache.hadoop.io.FloatWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.Map;
import java.util.HashMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.REALTIME_SKIP_MERGE_PROP;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.params.provider.Arguments.arguments;
public class TestHoodieRealtimeRecordReader {
private static final String PARTITION_COLUMN = "datestr";
private JobConf baseJobConf;
private FileSystem fs;
private Configuration hadoopConf;
@BeforeEach
public void setUp() {
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
baseJobConf = new JobConf(hadoopConf);
baseJobConf.set(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(1024 * 1024));
fs = FSUtils.getFs(basePath.toString(), baseJobConf);
}
@TempDir
public java.nio.file.Path basePath;
private Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
int numberOfRecords) throws InterruptedException, IOException {
return InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, fileId, baseCommit, newCommit,
numberOfRecords, 0,
0);
}
private void setHiveColumnNameProps(List<Schema.Field> fields, JobConf jobConf, boolean isPartitioned) {
String names = fields.stream().map(Field::name).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
String hiveOrderedColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase(PARTITION_COLUMN))
.map(Field::name).collect(Collectors.joining(","));
if (isPartitioned) {
hiveOrderedColumnNames += "," + PARTITION_COLUMN;
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, PARTITION_COLUMN);
}
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveOrderedColumnNames);
}
protected Properties getPropertiesForKeyGen() {
Properties properties = new Properties();
properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
properties.put("hoodie.datasource.write.recordkey.field", "_row_key");
properties.put("hoodie.datasource.write.partitionpath.field", "partition_path");
properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key");
properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path");
return properties;
}
@ParameterizedTest
@MethodSource("testArguments")
public void testReader(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled,
boolean partitioned) throws Exception {
testReaderInternal(diskMapType, isCompressionEnabled, partitioned);
}
@Test
public void testHFileInlineReader() throws Exception {
testReaderInternal(ExternalSpillableMap.DiskMapType.BITCASK, false, false,
HoodieLogBlock.HoodieLogBlockType.HFILE_DATA_BLOCK);
}
private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled,
boolean partitioned) throws Exception {
testReaderInternal(diskMapType, isCompressionEnabled, partitioned, HoodieLogBlock.HoodieLogBlockType.AVRO_DATA_BLOCK);
}
private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled,
boolean partitioned, HoodieLogBlock.HoodieLogBlockType logBlockType) throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ)
: InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
List<Pair<String, Integer>> logVersionsWithAction = new ArrayList<>();
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 1));
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 2));
// TODO: HUDI-154 Once Hive 2.x PR (PR-674) is merged, enable this change
// logVersionsWithAction.add(Pair.of(HoodieTimeline.ROLLBACK_ACTION, 3));
FileSlice fileSlice =
new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(basePath.toString()),
new Path(partitionDir.getAbsolutePath())) : "default", baseInstant, "fileid0");
logVersionsWithAction.forEach(logVersionWithAction -> {
try {
// update files or generate new log file
int logVersion = logVersionWithAction.getRight();
String action = logVersionWithAction.getKey();
int baseInstantTs = Integer.parseInt(baseInstant);
String instantTime = String.valueOf(baseInstantTs + logVersion);
String latestInstant =
action.equals(HoodieTimeline.ROLLBACK_ACTION) ? String.valueOf(baseInstantTs + logVersion - 2)
: instantTime;
HoodieLogFormat.Writer writer;
if (action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
writer = InputFormatTestUtil.writeRollback(partitionDir, fs, "fileid0", baseInstant, instantTime,
String.valueOf(baseInstantTs + logVersion - 1), logVersion);
} else {
writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", baseInstant,
instantTime, 120, 0, logVersion, logBlockType);
}
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
fileSlice.addLogFile(writer.getLogFile());
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.collect(Collectors.toList()),
instantTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), baseJobConf, null);
JobConf jobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, jobConf, partitioned);
jobConf.setEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType);
jobConf.setBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), isCompressionEnabled);
// validate record reader compaction
long logTmpFileStartTime = System.currentTimeMillis();
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file, merge in flight and return latest commit
// here all 100 records should be updated, see above
// another 20 new insert records should also output with new commit time.
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int recordCnt = 0;
while (recordReader.next(key, value)) {
Writable[] values = value.get();
// check if the record written is with latest commit, here "101"
assertEquals(latestInstant, values[0].toString());
key = recordReader.createKey();
value = recordReader.createValue();
recordCnt++;
}
recordReader.getPos();
assertEquals(1.0, recordReader.getProgress(), 0.05);
assertEquals(120, recordCnt);
recordReader.close();
// the temp file produced by logScanner should be deleted
assertTrue(!getLogTempFile(logTmpFileStartTime, System.currentTimeMillis(), diskMapType.toString()).exists());
} catch (Exception ioe) {
throw new HoodieException(ioe.getMessage(), ioe);
}
});
// Add Rollback last version to next log-file
}
private File getLogTempFile(long startTime, long endTime, String diskType) {
return Arrays.stream(new File("/tmp").listFiles())
.filter(f -> f.isDirectory() && f.getName().startsWith("hudi-" + diskType) && f.lastModified() > startTime && f.lastModified() < endTime)
.findFirst()
.orElse(new File(""));
}
@Test
public void testUnMergedReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
final int numRecords = 1000;
final int firstBatchLastRecordKey = numRecords - 1;
final int secondBatchLastRecordKey = 2 * numRecords - 1;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
// insert new records to log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numRecords, numRecords, 0);
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), baseJobConf, null);
JobConf jobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, jobConf, true);
// Enable merge skipping.
jobConf.set(REALTIME_SKIP_MERGE_PROP, "true");
// validate unmerged record reader
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file
// here all records should be present. Also ensure log records are in order.
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsAtCommit1 = 0;
int numRecordsAtCommit2 = 0;
Set<Integer> seenKeys = new HashSet<>();
int lastSeenKeyFromLog = firstBatchLastRecordKey;
while (recordReader.next(key, value)) {
Writable[] values = value.get();
String gotCommit = values[0].toString();
String keyStr = values[2].toString();
int gotKey = Integer.parseInt(keyStr.substring("key".length()));
if (gotCommit.equals(newCommitTime)) {
numRecordsAtCommit2++;
assertTrue(gotKey > firstBatchLastRecordKey);
assertTrue(gotKey <= secondBatchLastRecordKey);
assertEquals(gotKey, lastSeenKeyFromLog + 1);
lastSeenKeyFromLog++;
} else {
numRecordsAtCommit1++;
assertTrue(gotKey >= 0);
assertTrue(gotKey <= firstBatchLastRecordKey);
}
// Ensure unique key
assertFalse(seenKeys.contains(gotKey));
seenKeys.add(gotKey);
key = recordReader.createKey();
value = recordReader.createValue();
}
assertEquals(numRecords, numRecordsAtCommit1);
assertEquals(numRecords, numRecordsAtCommit2);
assertEquals(2 * numRecords, seenKeys.size());
assertEquals(1.0, recordReader.getProgress(), 0.05);
recordReader.close();
}
@ParameterizedTest
@MethodSource("testArguments")
public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled) throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords,
instantTime, HoodieTableType.MERGE_ON_READ);
InputFormatTestUtil.commit(basePath, instantTime);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
// update files or generate new log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
writeLogFile(partitionDir, schema, "fileid0", instantTime, newCommitTime, numberOfLogRecords);
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), baseJobConf, null);
JobConf jobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, jobConf, true);
jobConf.setEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType);
jobConf.setBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), isCompressionEnabled);
// validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file, merge in flight and return latest commit
// here the first 50 records should be updated, see above
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsRead = 0;
while (recordReader.next(key, value)) {
int currentRecordNo = numRecordsRead;
++numRecordsRead;
Writable[] values = value.get();
String recordCommitTime;
// check if the record written is with latest commit, here "101"
if (numRecordsRead > numberOfLogRecords) {
recordCommitTime = instantTime;
} else {
recordCommitTime = newCommitTime;
}
String recordCommitTimeSuffix = "@" + recordCommitTime;
assertEquals(values[0].toString(), recordCommitTime);
key = recordReader.createKey();
value = recordReader.createValue();
// Assert type STRING
assertEquals(values[5].toString(), "field" + currentRecordNo, "test value for field: field1");
assertEquals(values[6].toString(), "field" + currentRecordNo + recordCommitTimeSuffix,
"test value for field: field2");
assertEquals(values[7].toString(), "name" + currentRecordNo,
"test value for field: name");
// Assert type INT
IntWritable intWritable = (IntWritable) values[8];
assertEquals(intWritable.get(), currentRecordNo + recordCommitTime.hashCode(),
"test value for field: favoriteIntNumber");
// Assert type LONG
LongWritable longWritable = (LongWritable) values[9];
assertEquals(longWritable.get(), currentRecordNo + recordCommitTime.hashCode(),
"test value for field: favoriteNumber");
// Assert type FLOAT
FloatWritable floatWritable = (FloatWritable) values[10];
assertEquals(floatWritable.get(), (float) ((currentRecordNo + recordCommitTime.hashCode()) / 1024.0), 0,
"test value for field: favoriteFloatNumber");
// Assert type DOUBLE
DoubleWritable doubleWritable = (DoubleWritable) values[11];
assertEquals(doubleWritable.get(), (currentRecordNo + recordCommitTime.hashCode()) / 1024.0, 0,
"test value for field: favoriteDoubleNumber");
// Assert type MAP
ArrayWritable mapItem = (ArrayWritable) values[12];
Writable mapItemValue1 = mapItem.get()[0];
Writable mapItemValue2 = mapItem.get()[1];
assertEquals(((ArrayWritable) mapItemValue1).get()[0].toString(), "mapItem1",
"test value for field: tags");
assertEquals(((ArrayWritable) mapItemValue2).get()[0].toString(), "mapItem2",
"test value for field: tags");
assertEquals(((ArrayWritable) mapItemValue1).get().length, 2,
"test value for field: tags");
assertEquals(((ArrayWritable) mapItemValue2).get().length, 2,
"test value for field: tags");
Writable mapItemValue1value = ((ArrayWritable) mapItemValue1).get()[1];
Writable mapItemValue2value = ((ArrayWritable) mapItemValue2).get()[1];
assertEquals(((ArrayWritable) mapItemValue1value).get()[0].toString(), "item" + currentRecordNo,
"test value for field: tags[\"mapItem1\"].item1");
assertEquals(((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo,
"test value for field: tags[\"mapItem2\"].item1");
assertEquals(((ArrayWritable) mapItemValue1value).get()[1].toString(), "item" + currentRecordNo + recordCommitTimeSuffix,
"test value for field: tags[\"mapItem1\"].item2");
assertEquals(((ArrayWritable) mapItemValue2value).get()[1].toString(), "item2" + currentRecordNo + recordCommitTimeSuffix,
"test value for field: tags[\"mapItem2\"].item2");
// Assert type RECORD
ArrayWritable recordItem = (ArrayWritable) values[13];
Writable[] nestedRecord = recordItem.get();
assertFalse(((BooleanWritable) nestedRecord[0]).get(), "test value for field: testNestedRecord.isAdmin");
assertEquals(nestedRecord[1].toString(), "UserId" + currentRecordNo + recordCommitTimeSuffix,
"test value for field: testNestedRecord.userId");
// Assert type ARRAY
ArrayWritable arrayValue = (ArrayWritable) values[14];
Writable[] arrayValues = arrayValue.get();
for (int i = 0; i < arrayValues.length; i++) {
assertEquals("stringArray" + i + recordCommitTimeSuffix, arrayValues[i].toString(),
"test value for field: stringArray");
}
reader.close();
}
}
@ParameterizedTest
@MethodSource("testArguments")
public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMap.DiskMapType diskMapType,
boolean isCompressionEnabled) throws Exception {
// initial commit
List<HoodieLogFile> logFiles = new ArrayList<>();
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir =
InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords,
instantTime, HoodieTableType.MERGE_ON_READ);
InputFormatTestUtil.commit(basePath, instantTime);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
List<Field> firstSchemaFields = schema.getFields();
// update files and generate new log file but don't commit
schema = SchemaTestUtil.getComplexEvolvedSchema();
String newCommitTime = "101";
HoodieLogFormat.Writer writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numberOfLogRecords, 0, 1);
long size = writer.getCurrentSize();
logFiles.add(writer.getLogFile());
writer.close();
assertTrue(size > 0, "block - size should be > 0");
// write rollback for the previous block in new log file version
newCommitTime = "102";
writer = InputFormatTestUtil.writeRollbackBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime,
newCommitTime, "101", 1);
logFiles.add(writer.getLogFile());
writer.close();
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf),
basePath.toUri().toString(), logFiles, newCommitTime, Option.empty());
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), baseJobConf, null);
JobConf jobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
assertFalse(firstSchemaFields.containsAll(fields));
// Try to read all the fields passed by the new schema
setHiveColumnNameProps(fields, jobConf, true);
jobConf.setEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType);
jobConf.setBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), isCompressionEnabled);
HoodieRealtimeRecordReader recordReader;
try {
// validate record reader compaction
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
throw new RuntimeException("should've failed the previous line");
} catch (HoodieException e) {
// expected, field not found since the data written with the evolved schema was rolled back
}
// Try to read all the fields passed by the new schema
setHiveColumnNameProps(firstSchemaFields, jobConf, true);
// This time read only the fields which are part of parquet
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
while (recordReader.next(key, value)) {
// keep reading
}
reader.close();
}
private static Stream<Arguments> testArguments() {
// Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: partitioned
return Stream.of(
arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false),
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false),
arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, false),
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, false),
arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, true),
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, true),
arguments(ExternalSpillableMap.DiskMapType.BITCASK, true, true),
arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, true, true)
);
}
@Test
public void testIncrementalWithOnlylog() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String instantTime = "100";
final int numRecords = 1000;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime,
HoodieTableType.MERGE_ON_READ);
//FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
createDeltaCommitFile(basePath, instantTime,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
// insert new records to log file
try {
String newCommitTime = "102";
HoodieLogFormat.Writer writer =
InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime,
numRecords, numRecords, 0);
writer.close();
createDeltaCommitFile(basePath, newCommitTime,"2016/05/01", "2016/05/01/.fileid0_100.log.1_1-0-1", "fileid0");
InputFormatTestUtil.setupIncremental(baseJobConf, "101", 1);
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 1);
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
RecordReader<NullWritable, ArrayWritable> reader = inputFormat.getRecordReader(splits[0], newJobConf, Reporter.NULL);
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
// since we set incremental start commit as 101 and commit_number as 1.
// the data belong to commit 102 should be read out.
assertEquals(newCommitTime, values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
} catch (IOException e) {
throw new HoodieException(e.getMessage(), e);
}
}
@Test
public void testIncrementalWithReplace() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
//FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
InputFormatTestUtil.simulateInserts(partitionDir, ".parquet", "fileid1", 1, "200");
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileId = new ArrayList<>();
replacedFileId.add("fileid0");
partitionToReplaceFileIds.put("2016/05/01", replacedFileId);
createReplaceCommitFile(basePath,
"200","2016/05/01", "2016/05/01/fileid10_1-0-1_200.parquet", "fileid10", partitionToReplaceFileIds);
InputFormatTestUtil.setupIncremental(baseJobConf, "0", 1);
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 1);
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
newJobConf.set("columns.types", "string,string,string,string,string,string,string,string,bigint,string,string");
RecordReader<NullWritable, ArrayWritable> reader = inputFormat.getRecordReader(splits[0], newJobConf, Reporter.NULL);
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
// since we set incremental start commit as 0 and commit_number as 1.
// the data belong to commit 100 should be read out.
assertEquals("100", values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
}
private void createReplaceCommitFile(
java.nio.file.Path basePath,
String commitNumber,
String partitionPath,
String filePath,
String fileId,
Map<String, List<String>> partitionToReplaceFileIds) throws IOException {
List<HoodieWriteStat> writeStats = new ArrayList<>();
HoodieWriteStat writeStat = createHoodieWriteStat(basePath, commitNumber, partitionPath, filePath, fileId);
writeStats.add(writeStat);
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
replaceMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
writeStats.forEach(stat -> replaceMetadata.addWriteStat(partitionPath, stat));
File file = basePath.resolve(".hoodie").resolve(commitNumber + ".replacecommit").toFile();
file.createNewFile();
FileOutputStream fileOutputStream = new FileOutputStream(file);
fileOutputStream.write(replaceMetadata.toJsonString().getBytes(StandardCharsets.UTF_8));
fileOutputStream.flush();
fileOutputStream.close();
}
private HoodieWriteStat createHoodieWriteStat(java.nio.file.Path basePath, String commitNumber, String partitionPath, String filePath, String fileId) {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);
writeStat.setNumDeletes(0);
writeStat.setNumUpdateWrites(100);
writeStat.setNumWrites(100);
writeStat.setPath(filePath);
writeStat.setFileSizeInBytes(new File(new Path(basePath.toString(), filePath).toString()).length());
writeStat.setPartitionPath(partitionPath);
writeStat.setTotalLogFilesCompacted(100L);
HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats();
runtimeStats.setTotalScanTime(100);
runtimeStats.setTotalCreateTime(100);
runtimeStats.setTotalUpsertTime(100);
writeStat.setRuntimeStats(runtimeStats);
return writeStat;
}
private void createDeltaCommitFile(
java.nio.file.Path basePath,
String commitNumber,
String partitionPath,
String filePath,
String fileId) throws IOException {
List<HoodieWriteStat> writeStats = new ArrayList<>();
HoodieWriteStat writeStat = createHoodieWriteStat(basePath, commitNumber, partitionPath, filePath, fileId);
writeStats.add(writeStat);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
writeStats.forEach(stat -> commitMetadata.addWriteStat(partitionPath, stat));
File file = basePath.resolve(".hoodie").resolve(commitNumber + ".deltacommit").toFile();
file.createNewFile();
FileOutputStream fileOutputStream = new FileOutputStream(file);
fileOutputStream.write(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8));
fileOutputStream.flush();
fileOutputStream.close();
}
@Test
public void testLogOnlyReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant);
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
FileSlice fileSlice = new FileSlice("default", baseInstant, "fileid1");
try {
// update files or generate new log file
int logVersion = 1;
int baseInstantTs = Integer.parseInt(baseInstant);
String instantTime = String.valueOf(baseInstantTs + logVersion);
HoodieLogFormat.Writer writer = InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid1", baseInstant,
instantTime, 100, 0, logVersion);
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// create a split with new log file(s)
fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size));
RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()));
realtimeFileStatus.setMaxCommitTime(instantTime);
realtimeFileStatus.setBasePath(basePath.toString());
realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList()));
PathWithLogFilePath pathWithLogFileStatus = (PathWithLogFilePath) realtimeFileStatus.getPath();
BaseFileWithLogsSplit bs = pathWithLogFileStatus.buildSplit(pathWithLogFileStatus, 0, 0, new String[] {""});
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), Option.empty());
JobConf newJobConf = new JobConf(baseJobConf);
List<Schema.Field> fields = schema.getFields();
setHiveColumnNameProps(fields, newJobConf, false);
// create a dummy RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new HoodieRealtimeRecordReader(split, newJobConf, new HoodieEmptyRecordReader(split, newJobConf));
// use reader to read log file.
NullWritable key = reader.createKey();
ArrayWritable value = reader.createValue();
while (reader.next(key, value)) {
Writable[] values = value.get();
assertEquals(instantTime, values[0].toString());
key = reader.createKey();
value = reader.createValue();
}
reader.close();
} catch (Exception e) {
throw new HoodieException(e.getMessage(), e);
}
}
@Test
public void testIncrementalWithCompaction() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0");
// Add the paths
FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath());
createCompactionFile(basePath, "125");
// add inserts after compaction timestamp
InputFormatTestUtil.simulateInserts(partitionDir, ".parquet", "fileId2", 5, "200");
InputFormatTestUtil.commit(basePath, "200");
InputFormatTestUtil.setupIncremental(baseJobConf, "100", 10);
// verify that incremental reads do NOT show inserts after compaction timestamp
HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat();
inputFormat.setConf(baseJobConf);
InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1);
assertTrue(splits.length == 0);
}
private File createCompactionFile(java.nio.file.Path basePath, String commitTime)
throws IOException {
File file = basePath.resolve(".hoodie")
.resolve(HoodieTimeline.makeRequestedCompactionFileName(commitTime)).toFile();
assertTrue(file.createNewFile());
FileOutputStream os = new FileOutputStream(file);
try {
HoodieCompactionPlan compactionPlan = HoodieCompactionPlan.newBuilder().setVersion(2).build();
// Write empty commit metadata
os.write(TimelineMetadataUtils.serializeCompactionPlan(compactionPlan).get());
return file;
} finally {
os.close();
}
}
}