forked from apache/hudi
-
Notifications
You must be signed in to change notification settings - Fork 2
/
TestHoodieMergeOnReadTable.java
619 lines (527 loc) · 29 KB
/
TestHoodieMergeOnReadTable.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
/*
* 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.table;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableConfig;
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.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.Transformations;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.deltacommit.BaseSparkDeltaCommitActionExecutor;
import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
import org.apache.hudi.testutils.HoodieClientTestUtils;
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.storage.StorageLevel;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
import static org.apache.hudi.testutils.HoodieClientTestHarness.buildProfile;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness {
private HoodieTableMetaClient metaClient;
private HoodieTestDataGenerator dataGen;
@BeforeEach
void setUp() throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString());
metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties);
dataGen = new HoodieTestDataGenerator();
}
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
String newCommitTime = "001";
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
client.startCommitWithTime(newCommitTime);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
Map<String, String> allWriteStatusMergedMetadataMap =
MetadataMergeWriteStatus.mergeMetadataForWriteStatuses(statuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this
// should be 2 * records.size()
assertEquals(String.valueOf(2 * records.size()),
allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testUpsertPartitioner(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
* Write 1 (only inserts, written as base file)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient);
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("001", deltaCommit.get().getTimestamp(), "Delta commit should be 001");
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable);
BaseFileOnlyView roView = getHoodieTableFileSystemView(metaClient,
metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
Map<String, Long> fileIdToSize =
dataFilesToRead.collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize));
roView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
dataFilesToRead = roView.getLatestBaseFiles();
List<HoodieBaseFile> dataFilesList = dataFilesToRead.collect(Collectors.toList());
assertTrue(dataFilesList.size() > 0,
"Should list the base files we wrote in the delta commit");
/**
* Write 2 (only updates + inserts, written to .log file + correction of existing base file size)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> newRecords = dataGen.generateUpdates(newCommitTime, records);
newRecords.addAll(dataGen.generateInserts(newCommitTime, 20));
statuses = client.upsert(jsc().parallelize(newRecords), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = HoodieTableMetaClient.reload(metaClient);
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("002", deltaCommit.get().getTimestamp(), "Latest Delta commit should be 002");
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
allFiles = listAllBaseFilesInPath(hoodieTable);
roView = getHoodieTableFileSystemView(metaClient,
hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles);
dataFilesToRead = roView.getLatestBaseFiles();
List<HoodieBaseFile> newDataFilesList = dataFilesToRead.collect(Collectors.toList());
Map<String, Long> fileIdToNewSize =
newDataFilesList.stream().collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize));
assertTrue(fileIdToNewSize.entrySet().stream().anyMatch(entry -> fileIdToSize.get(entry.getKey()) < entry.getValue()));
List<String> inputPaths = roView.getLatestBaseFiles()
.map(baseFile -> new Path(baseFile.getPath()).getParent().toString())
.collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf(), inputPaths,
basePath(), new JobConf(hadoopConf()), true, false);
// Wrote 20 records in 2 batches
assertEquals(40, recordsRead.size(), "Must contain 40 records");
}
}
// TODO: Enable metadata virtual keys in this test once the feature HUDI-2593 is completed
@ParameterizedTest
@ValueSource(booleans = {false, true})
public void testLogFileCountsAfterCompaction(boolean preserveCommitMeta) throws Exception {
boolean populateMetaFields = true;
// insert 100 records
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true, false, HoodieIndex.IndexType.BLOOM,
1024 * 1024 * 1024L, HoodieClusteringConfig.newBuilder().build(), preserveCommitMeta);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig config = cfgBuilder.build();
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc().parallelize(records, 1);
writeClient.insert(recordsRDD, newCommitTime).collect();
// Update all the 100 records
newCommitTime = "101";
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc().parallelize(updatedRecords, 1);
HoodieReadClient readClient = new HoodieReadClient(context(), config);
JavaRDD<HoodieRecord> updatedTaggedRecordsRDD = readClient.tagLocation(updatedRecordsRDD);
writeClient.startCommitWithTime(newCommitTime);
writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect();
// Write them to corresponding avro logfiles
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(
writeClient.getEngineContext().getHadoopConf().get(), config, writeClient.getEngineContext());
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable
.of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter);
Set<String> allPartitions = updatedRecords.stream()
.map(record -> record.getPartitionPath())
.collect(Collectors.groupingBy(partitionPath -> partitionPath))
.keySet();
assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length);
// Verify that all data file has one log file
HoodieTable table = HoodieSparkTable.create(config, context(), metaClient);
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles =
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
for (FileSlice fileSlice : groupedLogFiles) {
assertEquals(1, fileSlice.getLogFiles().count(),
"There should be 1 log file written for the latest data file - " + fileSlice);
}
}
// Do a compaction
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = writeClient.compact(compactionInstantTime);
// Verify that recently written compacted data file has no log file
metaClient = HoodieTableMetaClient.reload(metaClient);
table = HoodieSparkTable.create(config, context(), metaClient);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
assertTrue(HoodieTimeline
.compareTimestamps(timeline.lastInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, newCommitTime),
"Compaction commit should be > than last insert");
for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles =
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
for (FileSlice slice : groupedLogFiles) {
assertEquals(0, slice.getLogFiles().count(), "After compaction there should be no log files visible on a full view");
}
assertTrue(result.getCommitMetadata().get().getWritePartitionPaths().stream().anyMatch(part -> part.contentEquals(partitionPath)));
}
// Check the entire dataset has all records still
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
for (int i = 0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath(), dataGen.getPartitionPaths()[i]);
}
Dataset<Row> actual = HoodieClientTestUtils.read(jsc(), basePath(), sqlContext(), fs(), fullPartitionPaths);
List<Row> rows = actual.collectAsList();
assertEquals(updatedRecords.size(), rows.size());
for (Row row: rows) {
assertEquals(row.getAs(HoodieRecord.COMMIT_TIME_METADATA_FIELD), preserveCommitMeta ? newCommitTime : compactionInstantTime);
}
}
}
/**
* Test to ensure metadata stats are correctly written to metadata file.
*/
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception {
HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY)
.withAutoCommit(false).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
HoodieTable table = HoodieSparkTable.create(cfg, context(), metaClient);
// Create a commit without metadata stats in metadata to test backwards compatibility
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getMetaClient().getCommitActionType();
HoodieInstant instant = new HoodieInstant(State.REQUESTED, commitActionType, "000");
activeTimeline.createNewInstant(instant);
activeTimeline.transitionRequestedToInflight(instant, Option.empty());
instant = new HoodieInstant(State.INFLIGHT, commitActionType, "000");
activeTimeline.saveAsComplete(instant, Option.empty());
String instantTime = "001";
client.startCommitWithTime(instantTime);
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, instantTime);
assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
// Read from commit file
table = HoodieSparkTable.create(cfg, context());
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline().getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
int inserts = 0;
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : pstat.getValue()) {
inserts += stat.getNumInserts();
}
}
assertEquals(200, inserts);
instantTime = "002";
client.startCommitWithTime(instantTime);
records = dataGen.generateUpdates(instantTime, records);
writeRecords = jsc().parallelize(records, 1);
statuses = client.upsert(writeRecords, instantTime);
//assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
inserts = 0;
int upserts = 0;
List<WriteStatus> writeStatusList = statuses.collect();
for (WriteStatus ws: writeStatusList) {
inserts += ws.getStat().getNumInserts();
upserts += ws.getStat().getNumUpdateWrites();
}
// Read from commit file
assertEquals(0, inserts);
assertEquals(200, upserts);
client.rollback(instantTime);
// Read from commit file
table = HoodieSparkTable.create(cfg, context());
metadata = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline()
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
inserts = 0;
upserts = 0;
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : pstat.getValue()) {
inserts += stat.getNumInserts();
upserts += stat.getNumUpdateWrites();
}
}
assertEquals(200, inserts);
assertEquals(0, upserts);
}
}
/**
* Test to ensure rolling stats are correctly written to the metadata file, identifies small files and corrects them.
*/
@Test
public void testRollingStatsWithSmallFileHandling() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
String instantTime = "000";
client.startCommitWithTime(instantTime);
List<HoodieRecord> records = dataGen.generateInserts(instantTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
JavaRDD<WriteStatus> statuses = client.insert(writeRecords, instantTime);
assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
// Read from commit file
HoodieTable table = HoodieSparkTable.create(cfg, context());
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline()
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
int inserts = 0;
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : pstat.getValue()) {
inserts += stat.getNumInserts();
fileIdToInsertsMap.put(stat.getFileId(), stat.getNumInserts());
fileIdToUpsertsMap.put(stat.getFileId(), stat.getNumUpdateWrites());
}
}
assertEquals(200, inserts);
instantTime = "001";
client.startCommitWithTime(instantTime);
// generate updates + inserts. inserts should be handled into small files
records = dataGen.generateUpdates(instantTime, records);
records.addAll(dataGen.generateInserts(instantTime, 200));
writeRecords = jsc().parallelize(records, 1);
statuses = client.upsert(writeRecords, instantTime);
assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
// Read from commit file
table = HoodieSparkTable.create(cfg, context());
metadata = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline()
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
inserts = 0;
int upserts = 0;
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : pstat.getValue()) {
assertTrue(fileIdToInsertsMap.containsKey(stat.getFileId()));
assertTrue(fileIdToUpsertsMap.containsKey(stat.getFileId()));
inserts += stat.getNumInserts();
upserts += stat.getNumUpdateWrites();
}
}
assertEquals(200, inserts);
assertEquals(200, upserts);
// Test small file handling after compaction
instantTime = "002";
client.scheduleCompactionAtInstant(instantTime, Option.of(metadata.getExtraMetadata()));
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(instantTime);
statuses = compactionMetadata.getWriteStatuses();
client.commitCompaction(instantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
// Read from commit file
table = HoodieSparkTable.create(cfg, context());
HoodieCommitMetadata metadata1 = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline()
.getInstantDetails(table.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
// Ensure that the metadata stats from the extra metadata of delta commits is copied over to the compaction commit
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
assertTrue(metadata1.getPartitionToWriteStats().containsKey(pstat.getKey()));
assertEquals(metadata1.getPartitionToWriteStats().get(pstat.getKey()).size(),
pstat.getValue().size());
}
// Write inserts + updates
instantTime = "003";
client.startCommitWithTime(instantTime);
// generate updates + inserts. inserts should be handled into small files
records = dataGen.generateUpdates(instantTime, records);
records.addAll(dataGen.generateInserts(instantTime, 200));
writeRecords = jsc().parallelize(records, 1);
statuses = client.upsert(writeRecords, instantTime);
assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
// Read from commit file
table = HoodieSparkTable.create(cfg, context());
metadata = HoodieCommitMetadata.fromBytes(
table.getActiveTimeline()
.getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
HoodieCommitMetadata.class);
inserts = 0;
upserts = 0;
for (Map.Entry<String, List<HoodieWriteStat>> pstat : metadata.getPartitionToWriteStats().entrySet()) {
for (HoodieWriteStat stat : pstat.getValue()) {
assertTrue(fileIdToInsertsMap.containsKey(stat.getFileId()));
inserts += stat.getNumInserts();
upserts += stat.getNumUpdateWrites();
}
}
assertEquals(200, inserts);
assertEquals(400, upserts);
}
}
/**
* Test to validate invoking table.handleUpdate() with input records from multiple partitions will fail.
*/
@Test
public void testHandleUpdateWithMultiplePartitions() throws Exception {
HoodieWriteConfig cfg = getConfig(true);
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
* Write 1 (only inserts, written as base file)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieSparkMergeOnReadTable hoodieTable = (HoodieSparkMergeOnReadTable) HoodieSparkTable.create(cfg, context(), metaClient);
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("001", deltaCommit.get().getTimestamp(), "Delta commit should be 001");
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertFalse(commit.isPresent());
FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable);
BaseFileOnlyView roView =
getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
assertFalse(dataFilesToRead.findAny().isPresent());
roView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
dataFilesToRead = roView.getLatestBaseFiles();
assertTrue(dataFilesToRead.findAny().isPresent(),
"should list the base files we wrote in the delta commit");
/**
* Write 2 (only updates, written to .log file)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
metaClient.reloadActiveTimeline();
records = dataGen.generateUpdates(newCommitTime, records);
writeRecords = jsc().parallelize(records, 1);
statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
/**
* Write 3 (only deletes, written to .log file)
*/
final String newDeleteTime = "004";
final String partitionPath = records.get(0).getPartitionPath();
final String fileId = statuses.get(0).getFileId();
client.startCommitWithTime(newDeleteTime);
metaClient.reloadActiveTimeline();
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletesFromExistingRecords(records);
JavaRDD<HoodieRecord> deleteRDD = jsc().parallelize(fewRecordsForDelete, 1);
// initialize partitioner
hoodieTable.getHoodieView().sync();
BaseSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable,
newDeleteTime, HoodieJavaRDD.of(deleteRDD));
actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD)));
final List<List<WriteStatus>> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> {
return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
}).map(Transformations::flatten).collect();
// Verify there are errors because records are from multiple partitions (but handleUpdate is invoked for
// specific partition)
WriteStatus status = deleteStatus.get(0).get(0);
assertTrue(status.hasErrors());
long numRecordsInPartition = fewRecordsForDelete.stream().filter(u ->
u.getPartitionPath().equals(partitionPath)).count();
assertEquals(fewRecordsForDelete.size() - numRecordsInPartition, status.getTotalErrorRecords());
}
}
@Test
public void testReleaseResource() throws Exception {
HoodieWriteConfig.Builder builder = getConfigBuilder(true);
builder.withReleaseResourceEnabled(true);
builder.withAutoCommit(false);
/**
* Write 1 (test when RELEASE_RESOURCE_ENABLE is true)
*/
try (SparkRDDWriteClient client = getHoodieWriteClient(builder.build())) {
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
writeRecords.persist(StorageLevel.MEMORY_AND_DISK());
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
client.commitStats(newCommitTime, statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType());
assertEquals(spark().sparkContext().persistentRdds().size(), 0);
}
builder.withReleaseResourceEnabled(false);
/**
* Write 2 (test when RELEASE_RESOURCE_ENABLE is false)
*/
try (SparkRDDWriteClient client = getHoodieWriteClient(builder.build())) {
String newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
writeRecords.persist(StorageLevel.MEMORY_AND_DISK());
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
client.commitStats(newCommitTime, statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType());
assertTrue(spark().sparkContext().persistentRdds().size() > 0);
}
}
}