Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -410,7 +410,37 @@ public void testDeletes() throws Exception {
}

/**
* Tests when update partition path is set in global bloom, existing record in old partition is deleted appropriately.
* When records getting inserted are deleted in the same write batch, hudi should have deleted those records and
* not be available in read path.
* @throws Exception
*/
@Test
public void testDeletesForInsertsInSameBatch() throws Exception {
HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);

/**
* Write 200 inserts and issue deletes to a subset(50) of inserts.
*/
String initCommitTime = "000";
String newCommitTime = "001";

final List<HoodieRecord> recordsInFirstBatch = new ArrayList<>();
Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
(String instantTime, Integer numRecordsInThisCommit) -> {
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(instantTime, 200);
List<HoodieRecord> fewRecordsForDelete = fewRecordsForInsert.subList(40, 90);

recordsInFirstBatch.addAll(fewRecordsForInsert);
recordsInFirstBatch.addAll(dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete));
return recordsInFirstBatch;
};

writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
-1, recordGenFunction, HoodieWriteClient::upsert, true, 150, 150, 1);
}

/**
* Test update of a record to different partition with Global Index.
*/
@ParameterizedTest
@EnumSource(value = IndexType.class, names = {"GLOBAL_BLOOM", "GLOBAL_SIMPLE"})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ public TestRawTripPayload generatePayloadForShortTripSchema(HoodieKey key, Strin
public static TestRawTripPayload generateRandomDeleteValue(HoodieKey key, String instantTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + instantTime, "driver-" + instantTime, 0.0,
true, false);
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
return new TestRawTripPayload(Option.of(rec.toString()), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA, true);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,11 @@ public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
}
}

public IndexedRecord getRecordToInsert(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter();
return jsonConverter.convert(getJsonData(), schema);
}

@Override
public Option<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,24 +59,28 @@ public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload

@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
return getInsertValue(schema);
}

Option<IndexedRecord> recordOption = getInsertValue(schema);
if (!recordOption.isPresent()) {
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (recordBytes.length == 0) {
return Option.empty();
}

GenericRecord genericRecord = (GenericRecord) recordOption.get();
// combining strategy here trivially ignores currentValue on disk and writes this record
Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
if (deleteMarker instanceof Boolean && (boolean) deleteMarker) {
IndexedRecord indexedRecord = HoodieAvroUtils.bytesToAvro(recordBytes, schema);
if (isDeleteRecord((GenericRecord) indexedRecord)) {
return Option.empty();
} else {
return Option.of(genericRecord);
return Option.of(indexedRecord);
}
}

@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return recordBytes.length == 0 ? Option.empty() : Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
/**
* @param genericRecord instance of {@link GenericRecord} of interest.
* @returns {@code true} if record represents a delete record. {@code false} otherwise.
*/
private boolean isDeleteRecord(GenericRecord genericRecord) {
Object deleteMarker = genericRecord.get("_hoodie_is_deleted");
return (deleteMarker instanceof Boolean && (boolean) deleteMarker);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hudi.common.model;

import org.apache.avro.Schema;
import org.apache.avro.Schema.Type;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

import java.io.IOException;
import java.util.Arrays;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;

/**
* Unit tests {@link OverwriteWithLatestAvroPayload}.
*/
public class TestOverwriteWithLatestAvroPayload {

private Schema schema;

@BeforeEach
public void setUp() throws Exception {
schema = Schema.createRecord(Arrays.asList(
new Schema.Field("id", Schema.create(Schema.Type.STRING), "", null),
new Schema.Field("partition", Schema.create(Schema.Type.STRING), "", null),
new Schema.Field("ts", Schema.create(Schema.Type.LONG), "", null),
new Schema.Field("_hoodie_is_deleted", Schema.create(Type.BOOLEAN), "", false)
));
}

@Test
public void testActiveRecords() throws IOException {
GenericRecord record1 = new GenericData.Record(schema);
record1.put("id", "1");
record1.put("partition", "partition0");
record1.put("ts", 0L);
record1.put("_hoodie_is_deleted", false);

GenericRecord record2 = new GenericData.Record(schema);
record2.put("id", "2");
record2.put("partition", "partition1");
record2.put("ts", 1L);
record2.put("_hoodie_is_deleted", false);

OverwriteWithLatestAvroPayload payload1 = new OverwriteWithLatestAvroPayload(record1, 1);
OverwriteWithLatestAvroPayload payload2 = new OverwriteWithLatestAvroPayload(record2, 2);
assertEquals(payload1.preCombine(payload2), payload2);
assertEquals(payload2.preCombine(payload1), payload2);

assertEquals(record1, payload1.getInsertValue(schema).get());
assertEquals(record2, payload2.getInsertValue(schema).get());

assertEquals(payload1.combineAndGetUpdateValue(record2, schema).get(), record1);
assertEquals(payload2.combineAndGetUpdateValue(record1, schema).get(), record2);
}

@Test
public void testDeletedRecord() throws IOException {
GenericRecord record1 = new GenericData.Record(schema);
record1.put("id", "1");
record1.put("partition", "partition0");
record1.put("ts", 0L);
record1.put("_hoodie_is_deleted", false);

GenericRecord delRecord1 = new GenericData.Record(schema);
delRecord1.put("id", "2");
delRecord1.put("partition", "partition1");
delRecord1.put("ts", 1L);
delRecord1.put("_hoodie_is_deleted", true);

OverwriteWithLatestAvroPayload payload1 = new OverwriteWithLatestAvroPayload(record1, 1);
OverwriteWithLatestAvroPayload payload2 = new OverwriteWithLatestAvroPayload(delRecord1, 2);
assertEquals(payload1.preCombine(payload2), payload2);
assertEquals(payload2.preCombine(payload1), payload2);

assertEquals(record1, payload1.getInsertValue(schema).get());
assertFalse(payload2.getInsertValue(schema).isPresent());

assertEquals(payload1.combineAndGetUpdateValue(delRecord1, schema).get(), record1);
assertFalse(payload2.combineAndGetUpdateValue(record1, schema).isPresent());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,15 @@

import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.RocksDBBasedMap;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.testutils.HoodieTestDataGenerator;
import org.apache.hudi.testutils.TestRawTripPayload;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.AvroSource;
import org.apache.hudi.utilities.testutils.sources.config.SourceConfigs;

import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
Expand Down Expand Up @@ -77,12 +76,12 @@ public static void resetDataGen() {
}

protected AbstractBaseTestSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
}

protected static Stream<GenericRecord> fetchNextBatch(TypedProperties props, int sourceLimit, String instantTime,
int partition) {
int partition) {
int maxUniqueKeys =
props.getInteger(SourceConfigs.MAX_UNIQUE_RECORDS_PROP, SourceConfigs.DEFAULT_MAX_UNIQUE_RECORDS);

Expand Down Expand Up @@ -119,11 +118,11 @@ protected static Stream<GenericRecord> fetchNextBatch(TypedProperties props, int
// if we generate update followed by deletes -> some keys in update batch might be picked up for deletes. Hence generating delete batch followed by updates
deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50).map(AbstractBaseTestSource::toGenericRecord);
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
.map(AbstractBaseTestSource::toGenericRecord);
} else {
LOG.info("After adjustments => NumInserts=" + numInserts + ", NumUpdates=" + numUpdates + ", maxUniqueRecords=" + maxUniqueKeys);
updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
.map(AbstractBaseTestSource::toGenericRecord);
}
Stream<GenericRecord> insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.map(AbstractBaseTestSource::toGenericRecord);
Expand All @@ -132,8 +131,8 @@ protected static Stream<GenericRecord> fetchNextBatch(TypedProperties props, int

private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord) {
try {
Option<IndexedRecord> recordOpt = hoodieRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
return (GenericRecord) recordOpt.get();
TestRawTripPayload payload = (TestRawTripPayload) hoodieRecord.getData();
return (GenericRecord) payload.getRecordToInsert(HoodieTestDataGenerator.AVRO_SCHEMA);
} catch (IOException e) {
return null;
}
Expand Down