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 @@ -132,7 +132,9 @@ public Iterator<HoodieRecord<T>> call(Integer partitionNum,
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
if (recordLocationMap.containsKey(rec.getKey())) {
rec.unseal();
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
rec.seal();
}
taggedRecords.add(rec);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,9 @@ HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord,
// currentLocation 2 times and it will fail the second time. So creating a new in memory
// copy of the hoodie record.
record = new HoodieRecord<>(inputRecord);
record.unseal();
record.setCurrentLocation(location.get());
record.seal();
}
return record;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,9 @@ Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient)
currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
currentRecord.unseal();
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
currentRecord.seal();
taggedRecords.add(currentRecord);
// the key from Result and the key being processed should be same
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,9 @@ private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTim

private void writeToBuffer(HoodieRecord<T> record) {
// update the new location of the record, so we know where to find it next
record.unseal();
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
record.seal();
Option<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,9 @@ public void write(HoodieRecord record, Option<IndexedRecord> avroRecord) {
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) avroRecord.get());
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, record);
// update the new location of record, so we know where to find it next
record.unseal();
record.setNewLocation(new HoodieRecordLocation(instantTime, writeStatus.getFileId()));
record.seal();
recordsWritten++;
insertRecordsWritten++;
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,9 @@ private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
HoodieRecord<T> record = newRecordsItr.next();
partitionPath = record.getPartitionPath();
// update the new location of the record, so we know where to find it next
record.unseal();
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
record.seal();
//NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
keyToNewRecords.put(record.getRecordKey(), record);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,9 @@ public void testSchemaEvolutionOnUpdate() throws Exception {
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
record1.unseal();
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
record1.seal();
updateRecords.add(record1);

try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,9 @@ public void testUpdateRecords() throws Exception {
TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1);
HoodieRecord updatedRecord1 = new HoodieRecord(
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
updatedRecord1.unseal();
updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
updatedRecord1.seal();

TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord insertedRecord1 = new HoodieRecord(
Expand Down Expand Up @@ -407,7 +409,9 @@ private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) {
updateRec.unseal();
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
updateRec.seal();
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,17 +63,24 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
*/
private HoodieRecordLocation newLocation;

/**
* Indicates whether the object is sealed.
*/
private boolean sealed;

public HoodieRecord(HoodieKey key, T data) {
this.key = key;
this.data = data;
this.currentLocation = null;
this.newLocation = null;
this.sealed = false;
}

public HoodieRecord(HoodieRecord<T> record) {
this(record.key, record.data);
this.currentLocation = record.currentLocation;
this.newLocation = record.newLocation;
this.sealed = record.sealed;
}

public HoodieKey getKey() {
Expand All @@ -100,6 +107,7 @@ public void deflate() {
* Sets the current currentLocation of the record. This should happen exactly-once
*/
public HoodieRecord setCurrentLocation(HoodieRecordLocation location) {
checkState();
assert currentLocation == null;
this.currentLocation = location;
return this;
Expand All @@ -114,6 +122,7 @@ public HoodieRecordLocation getCurrentLocation() {
* exactly-once.
*/
public HoodieRecord setNewLocation(HoodieRecordLocation location) {
checkState();
assert newLocation == null;
this.newLocation = location;
return this;
Expand Down Expand Up @@ -170,4 +179,18 @@ public String getRecordKey() {
assert key != null;
return key.getRecordKey();
}

public void seal() {
this.sealed = true;
}

public void unseal() {
this.sealed = false;
}

public void checkState() {
if (sealed) {
throw new UnsupportedOperationException("Not allowed to modify after sealed");
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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 static org.junit.Assert.fail;

import java.util.List;
import java.util.UUID;
import java.util.stream.Collectors;

import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

/**
* Tests for {@link HoodieRecord}.
*/
public class TestHoodieRecord {

private HoodieRecord hoodieRecord;

@Before
public void setUp() throws Exception {
final List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
final List<HoodieRecord> hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
hoodieRecord = hoodieRecords.get(0);
}

@Test
public void testModificationAfterSeal() {
hoodieRecord.seal();
final HoodieRecordLocation location = new HoodieRecordLocation("100", "0");
try {
hoodieRecord.setCurrentLocation(location);
fail("should fail since modification after sealed is not allowed");
} catch (Exception e) {
Assert.assertTrue(e instanceof UnsupportedOperationException);
}
}

@Test
public void testNormalModification() {
hoodieRecord.unseal();
final HoodieRecordLocation location = new HoodieRecordLocation("100", "0");
hoodieRecord.setCurrentLocation(location);
hoodieRecord.seal();

hoodieRecord.unseal();
hoodieRecord.setNewLocation(location);
hoodieRecord.seal();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,9 @@ public static List<String> upsertRecords(List<IndexedRecord> iRecords,
recordKeys.add(key);
HoodieRecord record = new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Option.of((GenericRecord) r)));
record.unseal();
record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID"));
record.seal();
records.put(key, record);
});
return recordKeys;
Expand Down